@@ -1701,6 +1701,12 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1701
1701
goto inner_err ;
1702
1702
}
1703
1703
1704
+ /* Treat key=Py_None as if it were never set */
1705
+ if (vo == Py_None ) {
1706
+ PyDict_DelItem (confdict , ko );
1707
+ Py_DECREF (ks );
1708
+ continue ;
1709
+ }
1704
1710
k = cfl_PyUnistr_AsUTF8 (ks , & ks8 );
1705
1711
if (!strcmp (k , "error_cb" )) {
1706
1712
if (!PyCallable_Check (vo )) {
@@ -1713,10 +1719,10 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1713
1719
Py_DECREF (h -> error_cb );
1714
1720
h -> error_cb = NULL ;
1715
1721
}
1716
- if ( vo != Py_None ) {
1717
- h -> error_cb = vo ;
1718
- Py_INCREF (h -> error_cb );
1719
- }
1722
+
1723
+ h -> error_cb = vo ;
1724
+ Py_INCREF (h -> error_cb );
1725
+
1720
1726
Py_XDECREF (ks8 );
1721
1727
Py_DECREF (ks );
1722
1728
continue ;
@@ -1731,10 +1737,10 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1731
1737
Py_DECREF (h -> throttle_cb );
1732
1738
h -> throttle_cb = NULL ;
1733
1739
}
1734
- if ( vo != Py_None ) {
1735
- h -> throttle_cb = vo ;
1736
- Py_INCREF (h -> throttle_cb );
1737
- }
1740
+
1741
+ h -> throttle_cb = vo ;
1742
+ Py_INCREF (h -> throttle_cb );
1743
+
1738
1744
Py_XDECREF (ks8 );
1739
1745
Py_DECREF (ks );
1740
1746
continue ;
@@ -1750,10 +1756,10 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1750
1756
Py_DECREF (h -> stats_cb );
1751
1757
h -> stats_cb = NULL ;
1752
1758
}
1753
- if ( vo != Py_None ) {
1754
- h -> stats_cb = vo ;
1755
- Py_INCREF (h -> stats_cb );
1756
- }
1759
+
1760
+ h -> stats_cb = vo ;
1761
+ Py_INCREF (h -> stats_cb );
1762
+
1757
1763
Py_XDECREF (ks8 );
1758
1764
Py_DECREF (ks );
1759
1765
continue ;
@@ -1763,10 +1769,9 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1763
1769
h -> logger = NULL ;
1764
1770
}
1765
1771
1766
- if (vo != Py_None ) {
1767
- h -> logger = vo ;
1768
- Py_INCREF (h -> logger );
1769
- }
1772
+ h -> logger = vo ;
1773
+ Py_INCREF (h -> logger );
1774
+
1770
1775
Py_XDECREF (ks8 );
1771
1776
Py_DECREF (ks );
1772
1777
continue ;
@@ -1789,18 +1794,14 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1789
1794
/*
1790
1795
* Pass configuration property through to librdkafka.
1791
1796
*/
1792
- if (vo == Py_None ) {
1793
- v = NULL ;
1794
- } else {
1795
- if (!(vs = cfl_PyObject_Unistr (vo ))) {
1796
- PyErr_SetString (PyExc_TypeError ,
1797
- "expected configuration "
1798
- "property value as type "
1799
- "unicode string" );
1800
- goto inner_err ;
1801
- }
1802
- v = cfl_PyUnistr_AsUTF8 (vs , & vs8 );
1797
+ if (!(vs = cfl_PyObject_Unistr (vo ))) {
1798
+ PyErr_SetString (PyExc_TypeError ,
1799
+ "expected configuration "
1800
+ "property value as type "
1801
+ "unicode string" );
1802
+ goto inner_err ;
1803
1803
}
1804
+ v = cfl_PyUnistr_AsUTF8 (vs , & vs8 );
1804
1805
1805
1806
if (rd_kafka_conf_set (conf , k , v , errstr , sizeof (errstr )) !=
1806
1807
RD_KAFKA_CONF_OK ) {
@@ -2361,7 +2362,7 @@ static PyObject *_init_cimpl (void) {
2361
2362
PyModule_AddIntConstant (m , "OFFSET_END" , RD_KAFKA_OFFSET_END );
2362
2363
PyModule_AddIntConstant (m , "OFFSET_STORED" , RD_KAFKA_OFFSET_STORED );
2363
2364
PyModule_AddIntConstant (m , "OFFSET_INVALID" , RD_KAFKA_OFFSET_INVALID );
2364
-
2365
+ PyModule_AddIntConstant ( m , "PARTITION_UA" , RD_KAFKA_PARTITION_UA );
2365
2366
return m ;
2366
2367
}
2367
2368
0 commit comments