@@ -1506,7 +1506,6 @@ static int common_conf_set_special(PyObject *confdict, rd_kafka_conf_t *conf,
1506
1506
!= RD_KAFKA_CONF_OK ) {
1507
1507
cfl_PyErr_Format (RD_KAFKA_RESP_ERR__INVALID_ARG ,
1508
1508
"%s" , errstr );
1509
-
1510
1509
Py_DECREF (vs );
1511
1510
Py_XDECREF (vs8 );
1512
1511
return 0 ;
@@ -1585,8 +1584,8 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1585
1584
/*
1586
1585
* Set debug contexts first to capture all events including plugin loading
1587
1586
*/
1588
- if ((vo = PyDict_GetItemString (confdict , "debug" )))
1589
- if ( !common_conf_set_special (confdict , conf , "debug" , vo ))
1587
+ if ((vo = PyDict_GetItemString (confdict , "debug" )) &&
1588
+ !common_conf_set_special (confdict , conf , "debug" , vo ))
1590
1589
goto outer_err ;
1591
1590
1592
1591
/*
@@ -1598,40 +1597,18 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1598
1597
* correct order.
1599
1598
*/
1600
1599
if ((vo = PyDict_GetItemString (confdict , "plugin.library.paths" ))) {
1601
- const char * v ;
1602
- char errstr [256 ];
1600
+ /* Resolve plugin paths */
1603
1601
PyObject * resolved ;
1604
- PyObject * vs = NULL , * vs8 = NULL ;
1605
1602
1606
- /* Resolve plugin paths */
1607
1603
resolved = resolve_plugins (vo );
1608
1604
if (!resolved )
1609
1605
goto outer_err ;
1610
1606
1611
- if (!common_conf_set_special (confdict , conf , "plugin.library.paths" , vo )) {
1607
+ if (!common_conf_set_special (confdict , conf ,
1608
+ "plugin.library.paths" ,
1609
+ resolved )) {
1612
1610
Py_DECREF (resolved );
1613
- rd_kafka_conf_destroy (conf );
1614
- Py_DECREF (confdict );
1615
-
1616
- return NULL ;
1617
- }
1618
-
1619
- Py_DECREF (resolved );
1620
-
1621
- v = cfl_PyUnistr_AsUTF8 (vs , & vs8 );
1622
-
1623
- if (rd_kafka_conf_set (conf , "plugin.library.paths" , v , errstr , sizeof (errstr ))
1624
- != RD_KAFKA_CONF_OK ) {
1625
- cfl_PyErr_Format (RD_KAFKA_RESP_ERR__INVALID_ARG ,
1626
- "%s" , errstr );
1627
-
1628
- rd_kafka_conf_destroy (conf );
1629
- Py_DECREF (confdict );
1630
-
1631
- Py_XDECREF (vs8 );
1632
- Py_XDECREF (vs );
1633
-
1634
- return NULL ;
1611
+ goto outer_err ;
1635
1612
}
1636
1613
Py_DECREF (resolved );
1637
1614
}
@@ -1643,16 +1620,15 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1643
1620
"set default topic configuration values in the global dict");
1644
1621
*/
1645
1622
if (PyDict_Update (confdict , vo ) == -1 ) {
1646
- rd_kafka_conf_destroy (conf );
1647
- Py_DECREF (confdict );
1648
- return NULL ;
1623
+ goto outer_err ;
1649
1624
}
1650
1625
PyDict_DelItemString (confdict , "default.topic.config" );
1651
1626
}
1652
1627
1653
1628
/* Convert config dict to config key-value pairs. */
1654
1629
while (PyDict_Next (confdict , & pos , & ko , & vo )) {
1655
- PyObject * ks , * ks8 ;
1630
+ PyObject * ks ;
1631
+ PyObject * ks8 = NULL ;
1656
1632
PyObject * vs = NULL , * vs8 = NULL ;
1657
1633
const char * k ;
1658
1634
const char * v ;
@@ -1663,10 +1639,7 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1663
1639
PyErr_SetString (PyExc_TypeError ,
1664
1640
"expected configuration property name "
1665
1641
"as type unicode string" );
1666
- rd_kafka_conf_destroy (conf );
1667
- Py_DECREF (confdict );
1668
-
1669
- return NULL ;
1642
+ goto inner_err ;
1670
1643
}
1671
1644
1672
1645
k = cfl_PyUnistr_AsUTF8 (ks , & ks8 );
@@ -1675,13 +1648,7 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1675
1648
PyErr_SetString (PyExc_TypeError ,
1676
1649
"expected error_cb property "
1677
1650
"as a callable function" );
1678
- rd_kafka_conf_destroy (conf );
1679
- Py_DECREF (confdict );
1680
-
1681
- Py_XDECREF (ks8 );
1682
- Py_DECREF (ks );
1683
-
1684
- return NULL ;
1651
+ goto inner_err ;
1685
1652
}
1686
1653
if (h -> error_cb ) {
1687
1654
Py_DECREF (h -> error_cb );
@@ -1699,13 +1666,7 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1699
1666
PyErr_SetString (PyExc_ValueError ,
1700
1667
"expected throttle_cb property "
1701
1668
"as a callable function" );
1702
- rd_kafka_conf_destroy (conf );
1703
- Py_DECREF (confdict );
1704
-
1705
- Py_XDECREF (ks8 );
1706
- Py_DECREF (ks );
1707
-
1708
- return NULL ;
1669
+ goto inner_err ;
1709
1670
}
1710
1671
if (h -> throttle_cb ) {
1711
1672
Py_DECREF (h -> throttle_cb );
@@ -1723,13 +1684,7 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1723
1684
PyErr_SetString (PyExc_TypeError ,
1724
1685
"expected stats_cb property "
1725
1686
"as a callable function" );
1726
- rd_kafka_conf_destroy (conf );
1727
- Py_DECREF (confdict );
1728
-
1729
- Py_XDECREF (ks8 );
1730
- Py_DECREF (ks );
1731
-
1732
- return NULL ;
1687
+ goto inner_err ;
1733
1688
}
1734
1689
1735
1690
if (h -> stats_cb ) {
@@ -1765,13 +1720,7 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1765
1720
r = consumer_conf_set_special (h , conf , k , vo );
1766
1721
if (r == -1 ) {
1767
1722
/* Error */
1768
- Py_XDECREF (ks8 );
1769
- Py_DECREF (ks );
1770
- rd_kafka_conf_destroy (conf );
1771
- Py_DECREF (confdict );
1772
-
1773
- return NULL ;
1774
-
1723
+ goto inner_err ;
1775
1724
} else if (r == 1 ) {
1776
1725
/* Handled */
1777
1726
continue ;
@@ -1789,13 +1738,7 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1789
1738
"expected configuration "
1790
1739
"property value as type "
1791
1740
"unicode string" );
1792
- rd_kafka_conf_destroy (conf );
1793
- Py_DECREF (confdict );
1794
-
1795
- Py_XDECREF (ks8 );
1796
- Py_DECREF (ks );
1797
-
1798
- return NULL ;
1741
+ goto inner_err ;
1799
1742
}
1800
1743
v = cfl_PyUnistr_AsUTF8 (vs , & vs8 );
1801
1744
}
@@ -1804,15 +1747,7 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1804
1747
RD_KAFKA_CONF_OK ) {
1805
1748
cfl_PyErr_Format (RD_KAFKA_RESP_ERR__INVALID_ARG ,
1806
1749
"%s" , errstr );
1807
- rd_kafka_conf_destroy (conf );
1808
- Py_DECREF (confdict );
1809
-
1810
- Py_XDECREF (vs8 );
1811
- Py_XDECREF (vs );
1812
- Py_XDECREF (ks8 );
1813
- Py_DECREF (ks );
1814
-
1815
- return NULL ;
1750
+ goto inner_err ;
1816
1751
}
1817
1752
1818
1753
Py_XDECREF (vs8 );
@@ -1863,6 +1798,12 @@ rd_kafka_conf_t *common_conf_setup (rd_kafka_type_t ktype,
1863
1798
h -> initiated = 1 ;
1864
1799
1865
1800
return conf ;
1801
+
1802
+ outer_err :
1803
+ Py_DECREF (confdict );
1804
+ rd_kafka_conf_destroy (conf );
1805
+
1806
+ return NULL ;
1866
1807
}
1867
1808
1868
1809
0 commit comments