> On March 29, 2015, 7:33 p.m., Jun Rao wrote: > > clients/src/main/java/org/apache/kafka/common/protocol/ApiVersion.java, > > line 28 > > <https://reviews.apache.org/r/28769/diff/21/?file=908349#file908349line28> > > > > Since this is for intra-broker communication, should we move this class > > to core? > > Gwen Shapira wrote: > Sure. At first I thought we'll need it when we move all > requests/responses to o.a.k.common, but on second look, it doesn't seem like > the requests/responses themselves need it.
This was a bigger change than I expected, since Scala's enums are very limited compared to Java. > On March 29, 2015, 7:33 p.m., Jun Rao wrote: > > core/src/test/scala/unit/kafka/cluster/BrokerTest.scala, line 115 > > <https://reviews.apache.org/r/28769/diff/21/?file=908400#file908400line115> > > > > Is toString() customized on EndPoint or do you intend to use > > connectionString()? Also, if host is null, should we standardize the > > connection string to be PLAINTEXT://:9092? Ah, yes, this is confusing - EndPoints have toString() and BrokerEndPoints have connectionString(). For BrokerEndPoint, connectionString() makes more sense. a "natural" toString() would contain brokerId too. For consistency, I'd swap EndPoint to use connectionString() as well. Regarding "null", I agree that its better to be consistent and not print "null" anywhere. BrokerEndPoint should never have null anyway (since it represents an actual network connection, not a listner), but we need to handle this in EndPoint. > On March 29, 2015, 7:33 p.m., Jun Rao wrote: > > core/src/main/scala/kafka/server/KafkaConfig.scala, lines 822-829 > > <https://reviews.apache.org/r/28769/diff/21/?file=908381#file908381line822> > > > > Do we default to PLAINTEXT://null:6667 or PLAINTEXT://:9092? Do we have > > to explicitly deal with hostName being null? Fixed the doc. We don't need to deal with null here - host.name defaults to "", which is the right thing. We actually test for this (KafkaConfigTest.scala line 163) > On March 29, 2015, 7:33 p.m., Jun Rao wrote: > > core/src/main/scala/kafka/server/KafkaConfig.scala, line 840 > > <https://reviews.apache.org/r/28769/diff/21/?file=908381#file908381line840> > > > > Similar comment as the above, do we need to handle a null > > adverisedHostName explicitly? It will default to hostname. - Gwen ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/28769/#review78160 ----------------------------------------------------------- On March 27, 2015, 10:04 p.m., Gwen Shapira wrote: > > ----------------------------------------------------------- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/28769/ > ----------------------------------------------------------- > > (Updated March 27, 2015, 10:04 p.m.) > > > Review request for kafka. > > > Bugs: KAFKA-1809 > https://issues.apache.org/jira/browse/KAFKA-1809 > > > Repository: kafka > > > Description > ------- > > forgot rest of patch > > > merge with trunk > > > Diffs > ----- > > clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java > fa9daaef66ff7961e1c46cd0cd8fed18a53bccd8 > clients/src/main/java/org/apache/kafka/common/protocol/ApiVersion.java > PRE-CREATION > > clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java > PRE-CREATION > clients/src/main/java/org/apache/kafka/common/utils/Utils.java > 920b51a6c3c99639fbc9dc065637eeee3c19fabd > clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java > c899813d55b9c4786adde3d840f040d6645d27c8 > config/server.properties 1614260b71a658b405bb24157c8f12b1f1031aa5 > core/src/main/scala/kafka/admin/AdminUtils.scala > b700110f2d7f1ede235af55d8e37e1b5592c6c7d > core/src/main/scala/kafka/admin/TopicCommand.scala > f400b71f8444fffd3fc1d8398a283682390eba4e > core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala > 24aaf954dc42e2084454fa5fc9e8f388ea95c756 > core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala > 4ff7e8f8cc695551dd5d2fe65c74f6b6c571e340 > core/src/main/scala/kafka/api/TopicMetadata.scala > 0190076df0adf906ecd332284f222ff974b315fc > core/src/main/scala/kafka/api/TopicMetadataResponse.scala > 92ac4e687be22e4800199c0666bfac5e0059e5bb > core/src/main/scala/kafka/api/UpdateMetadataRequest.scala > 530982e36b17934b8cc5fb668075a5342e142c59 > core/src/main/scala/kafka/client/ClientUtils.scala > ebba87f0566684c796c26cb76c64b4640a5ccfde > core/src/main/scala/kafka/cluster/Broker.scala > 0060add008bb3bc4b0092f2173c469fce0120be6 > core/src/main/scala/kafka/cluster/BrokerEndPoint.scala PRE-CREATION > core/src/main/scala/kafka/cluster/EndPoint.scala PRE-CREATION > core/src/main/scala/kafka/cluster/SecurityProtocol.scala PRE-CREATION > core/src/main/scala/kafka/common/BrokerEndPointNotAvailableException.scala > PRE-CREATION > core/src/main/scala/kafka/consumer/ConsumerConfig.scala > 9ebbee6c16dc83767297c729d2d74ebbd063a993 > core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala > b9e2bea7b442a19bcebd1b350d39541a8c9dd068 > core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala > 152fda5d1dcdf319399fdeeb8457006090ebe56c > core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala > b1cf0db8741e384e4e4119751058ea87b2589e57 > core/src/main/scala/kafka/controller/ControllerChannelManager.scala > c582191636f6188c25d62a67ff0315b56f163133 > core/src/main/scala/kafka/controller/KafkaController.scala > 09fc46d759b74bcdad2d2a610d9c5a93ff02423f > core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala > d281bb31a66fd749ecddfbe38479b6903f436831 > core/src/main/scala/kafka/javaapi/TopicMetadata.scala > f384e04678df10a5b46a439f475c63371bf8e32b > core/src/main/scala/kafka/network/RequestChannel.scala > 7b1db3dbbb2c0676f166890f566c14aa248467ab > core/src/main/scala/kafka/network/SocketServer.scala > 76ce41aed6e04ac5ba88395c4d5008aca17f9a73 > core/src/main/scala/kafka/producer/ProducerPool.scala > 43df70bb461dd3e385e6b20396adef3c4016a3fc > core/src/main/scala/kafka/server/AbstractFetcherManager.scala > 20c00cb8cc2351950edbc8cb1752905a0c26e79f > core/src/main/scala/kafka/server/AbstractFetcherThread.scala > 5d5cf5897cc37b3595f14bfe9d7cde43456bcc4b > core/src/main/scala/kafka/server/KafkaApis.scala > c33e848256be6159a148319bf74bf1590f27cc06 > core/src/main/scala/kafka/server/KafkaConfig.scala > 422451aec5ea0442eb2e4c1ae772885b813904a9 > core/src/main/scala/kafka/server/KafkaHealthcheck.scala > 7907987e43404487382de7f4cc294f0d01ac15a7 > core/src/main/scala/kafka/server/KafkaServer.scala > 4db3384545be8c237d6fc9646716ab67d5193ec5 > core/src/main/scala/kafka/server/MetadataCache.scala > 6aef6e4508ecadbbcc1e12bed2054547b7aa333e > core/src/main/scala/kafka/server/ReplicaFetcherManager.scala > 351dbbad3bdb709937943b336a5b0a9e0162a5e2 > core/src/main/scala/kafka/server/ReplicaFetcherThread.scala > 96faa7b4ed7c9ba8a3f6f9f114bd94e19b3a7ac0 > core/src/main/scala/kafka/server/ReplicaManager.scala > 44f0026e6676d8d764dd59dbcc6bb7bb727a3ba6 > core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala > d1e7c434e77859d746b8dc68dd5d5a3740425e79 > core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala > ba6ddd7a909df79a0f7d45e8b4a2af94ea0fceb6 > core/src/main/scala/kafka/tools/SimpleConsumerShell.scala > b4f903b6c7c3bb725cac7c05eb1f885906413c4d > core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala > 111c9a8b94ce45d95551482e9fd3f8c1cccbf548 > core/src/main/scala/kafka/utils/Utils.scala > 738c1af9ef5de16fdf5130daab69757a14c48b5c > core/src/main/scala/kafka/utils/ZkUtils.scala > 7ae999ec619443d35a9cb8fbcd531fca0c51c8c0 > core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala > 7eb6d05b66b16f8b0ab64eb43dcc1a25168fa5be > core/src/test/scala/integration/kafka/api/ProducerSendTest.scala > 3df450784592b894008e7507b2737f9bb07f7bd2 > core/src/test/scala/other/kafka/TestOffsetManager.scala > a106379d96ab3cbdf54a9f162db44a6df177e733 > core/src/test/scala/unit/kafka/KafkaConfigTest.scala > 4d36b8b1173f60d43463c13c9d8c1275a84c8c28 > core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala > 8bc178505e00932a316c46ed1d904bd57b5b3f75 > core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala > 4cb803f3e86dbc97f1749b2db9788fe9d376d49a > core/src/test/scala/unit/kafka/cluster/BrokerTest.scala PRE-CREATION > core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala > 995397ba2e2dfc6fadd9d5c5efd90f2c4ac0d59c > core/src/test/scala/unit/kafka/integration/FetcherTest.scala > 3093e459935ecf8e5b34fca34a422674562a7034 > core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala > 062790f15d4ab6e0ee7a98bef12aaa2b22da3bfa > core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala > a671af4a87d5c2fb42ff48c553bca7cae6538231 > core/src/test/scala/unit/kafka/log/LogTest.scala > 3c0599cc2ea12566fbec8233e3a73b31854b3b38 > core/src/test/scala/unit/kafka/network/SocketServerTest.scala > 0af23abf146d99e3d6cf31e5d6b95a9e63318ddb > core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala > be90c5bc7f1f5ba8a237d1c7176f27029727c918 > core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala > 812df5989940095c1a347066136ab55153bdad21 > core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala > 296e2b563041318d360db4037411879613a63968 > core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala > 191251d1340b5e5b2d649b37af3c6c1896d07e6e > core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala > 7f47e6f9a74314ed9e9f19d0c97931f3f2e49259 > core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala > f2528059dd86bb051d1ce9d72039cde3b78b8817 > core/src/test/scala/unit/kafka/server/LogOffsetTest.scala > 8c9f9e748e888da6035755c9f4241ac3aa0500c9 > core/src/test/scala/unit/kafka/utils/TestUtils.scala > 1682a77362123449de6bb1d54a55887409990a24 > > system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json > 250ffe08ae4c2d35f7c79c883f47bbc27d5361b5 > > system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json > 3f9e7d05db17e170aea4fee5a13ab5534dd5db2e > > system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json > b10c6266a786acf0c4b1accb63f9727486d09fd9 > > system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json > 22c08023f3d9e7678fdb990518bdb7931b64b6ac > > system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json > 1317d8460f82c3ff09d5a13a1277d695ba71d6e7 > > system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json > d31346506a19dd066519303a19a191f70599918d > > system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json > bd00f130d7fc69dc43cf088e298abc43e51bf765 > > system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json > ac5c13b09a84f975e345b635cf44a76935251244 > > system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json > 6dadd7a288664ebdc7d023c54d9cdb5a54d0c2e7 > > system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json > 614cb1cf01d3c76d8a97c2b52fb52bd83ec80c4b > > system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json > fbf01772545f016c28448235a5859429a3a83780 > > system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json > 317a6e3d3384009668e822e6c7e38b3fa34fcd6e > > system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json > d1a790b34fa5b42fe9fce2dabf6833048f9905ea > > system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json > 8f4f8bfac71463bbe52906ee7a127d8b0b5f48a3 > > system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json > a307f855c944946724cf3a91a9c07cc7b57b71f6 > > system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json > 827319eb6c6dbba0c8000b6e5e5a657a112deb21 > > system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json > ae015cd0eb82b3d3f04e14c808b66dad702f7bf0 > > system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json > 1951a1220956f931032e8fcd214784d76472054b > system_test/replication_testsuite/testcase_1/testcase_1_properties.json > 0c6d7a316cc6b51ac0755ca03558507db0706c31 > system_test/run_all.sh PRE-CREATION > system_test/run_all_replica.sh PRE-CREATION > system_test/utils/kafka_system_test_utils.py > 41d511cbc310fa87e0f2cd2f772e479e8e3ae4e2 > > Diff: https://reviews.apache.org/r/28769/diff/ > > > Testing > ------- > > > Thanks, > > Gwen Shapira > >