[jira] [Commented] (KAFKA-6441) FetchRequest populates buffer of size MinBytes, even if response is smaller

2018-01-11 Thread Ivan Babrou (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-6441?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16323603#comment-16323603
 ] 

Ivan Babrou commented on KAFKA-6441:


I dumped raw bytes from Kafka responses and it seems like buffers are fully 
populated with messages. Digging deeper to find out what's causing Sarama to 
only read the first message.

> FetchRequest populates buffer of size MinBytes, even if response is smaller
> ---
>
> Key: KAFKA-6441
> URL: https://issues.apache.org/jira/browse/KAFKA-6441
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.11.0.1
>Reporter: Ivan Babrou
>
> We're using Sarama Go client as consumer, but I don't think it's relevant. 
> Producer is syslog-ng with Kafka output, I'm not quite sure which log format 
> Kafka itself is using, but I can assume 0.11.0.0, because that's what is set 
> in topic settings.
> Our FetchRequest has minSize = 16MB, maxSize = 64, maxWait = 500ms. For a 
> silly reason, Kafka decides to reply with at least minSize buffer with just 
> one 1KB log message. When Sarama was using older consumer API, everything was 
> okay. When we upgraded to 0.11.0.0 consumer API, consumer traffic for 
> 125Mbit/s topic spiked to 55000Mbit/s on the wire and consumer wasn't even 
> able to keep up.
> 1KB message in a 16MB buffer is 1,600,000% overhead.
> I don't think there's any valid reason to do this.
> It's also mildly annoying that there is no tag 0.11.0.1 in git, looking at 
> changes is harder than it should be.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-6443) KTable involved in multiple joins could result in duplicate results

2018-01-11 Thread Matthias J. Sax (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-6443?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16323403#comment-16323403
 ] 

Matthias J. Sax commented on KAFKA-6443:


It tend to think its a bug, because the intermediate result is wrong. We should 
never produce wrong results, even if we send an correct update record right 
after the wrong one. Thus, it's different to the "flush" issue, that produces 
the exact some update record twice -- this, is undesired by not wrong IMHO, 
while emitting a wrong record is definitely a bug from my point of view.

> KTable involved in multiple joins could result in duplicate results
> ---
>
> Key: KAFKA-6443
> URL: https://issues.apache.org/jira/browse/KAFKA-6443
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Reporter: Guozhang Wang
>
> Consider the following multi table-table joins:
> {code}
> table1.join(table2).join(table2);// "join" could be replaced with 
> "leftJoin" and "outerJoin"
> {code}
> where {{table2}} is involved multiple times in this multi-way joins. In this 
> case, when a new record from the source topic of {{table2}} is being 
> processing, it will send to two children down in the topology and hence may 
> resulting in duplicated join results depending on the join types.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Comment Edited] (KAFKA-6185) Selector memory leak with high likelihood of OOM in case of down conversion

2018-01-11 Thread Brett Rann (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-6185?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16253043#comment-16253043
 ] 

Brett Rann edited comment on KAFKA-6185 at 1/12/18 1:55 AM:


*update* the fix ends up in the client jar, so dropping it in like that didn't 
work.  Instead I've deployed the full kafka_2.11-1.0.1-SNAPSHOT.tgz that gets 
built and they're both now running on that.

Steps I took to build and deploy:

{noformat}
 $ gradle

Gradle 4.3


Build time:   2017-10-30 15:43:29 UTC
Revision: c684c202534c4138b51033b52d871939b8d38d72

Groovy:   2.4.12
Ant:  Apache Ant(TM) version 1.9.6 compiled on June 29 2015
JVM:  1.8.0_152 (Oracle Corporation 25.152-b16)
OS:   Mac OS X 10.12.6 x86_64

java version "1.8.0_152"
Java(TM) SE Runtime Environment (build 1.8.0_152-b16)
Java HotSpot(TM) 64-Bit Server VM (build 25.152-b16, mixed mode)
{noformat}

{noformat}
git clone g...@github.com:apache/kafka.git
git checkout 1.0
git branch -v
* 1.0   1a5a547bb KAFKA-6190: Use consumer.position() instead of 
record.offset() to advance in GlobalKTable restoration to avoid transactional 
control messages
trunk d04daf570 MINOR: Exclude Committer Checklist section from commit message
gradle
./gradlew clean releaseTarGz
{noformat}

note: came back to edit this message later to be more clear of the steps 
finally taken to deploy.


was (Author: brettrann):
*update* the fix ends up in the client jar, so dropping it in like that didn't 
work.  Instead I've deployed the full kafka_2.11-1.0.1-SNAPSHOT.tgz that gets 
built and they're both now running on that.

Our memory problem persists.

I have a .hprof from -XX:+HeapDumpOnOutOfMemoryError and a manual dump.bin 
which i could share privately. And they are still in an OOMing state if any 
further poking is required. We have jmx.rmi exposed if that's useful.

Steps I took to build and deploy:

{noformat}
 $ gradle

Gradle 4.3


Build time:   2017-10-30 15:43:29 UTC
Revision: c684c202534c4138b51033b52d871939b8d38d72

Groovy:   2.4.12
Ant:  Apache Ant(TM) version 1.9.6 compiled on June 29 2015
JVM:  1.8.0_152 (Oracle Corporation 25.152-b16)
OS:   Mac OS X 10.12.6 x86_64

java version "1.8.0_152"
Java(TM) SE Runtime Environment (build 1.8.0_152-b16)
Java HotSpot(TM) 64-Bit Server VM (build 25.152-b16, mixed mode)
{noformat}

{noformat}
git clone g...@github.com:apache/kafka.git
git checkout 1.0
git branch -v
* 1.0   1a5a547bb KAFKA-6190: Use consumer.position() instead of 
record.offset() to advance in GlobalKTable restoration to avoid transactional 
control messages
trunk d04daf570 MINOR: Exclude Committer Checklist section from commit message
gradle
./gradlew clean releaseTarGz
{noformat}
I then renamed ./core/build/libs/kafka_2.11-1.0.1-SNAPSHOT.jar to 
kafka_2.11-1.0.0.jar and dropped it into the lib directory where the affected 
kafka is installed and restarted kafka.



> Selector memory leak with high likelihood of OOM in case of down conversion
> ---
>
> Key: KAFKA-6185
> URL: https://issues.apache.org/jira/browse/KAFKA-6185
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 1.0.0
> Environment: Ubuntu 14.04.5 LTS
> 5 brokers: 1&2 on 1.0.0 3,4,5 on 0.11.0.1
> inter.broker.protocol.version=0.11.0.1
> log.message.format.version=0.11.0.1
> clients a mix of 0.9, 0.10, 0.11
>Reporter: Brett Rann
>Assignee: Rajini Sivaram
>Priority: Blocker
>  Labels: regression
> Fix For: 1.1.0, 1.0.1
>
> Attachments: Kafka_Internals___Datadog.png, 
> Kafka_Internals___Datadog.png
>
>
> We are testing 1.0.0 in a couple of environments.
> Both have about 5 brokers, with two 1.0.0 brokers and the rest 0.11.0.1 
> brokers.
> One is using on disk message format 0.9.0.1, the other 0.11.0.1
> we have 0.9, 0.10, and 0.11 clients connecting.
> The cluster on the 0.9.0.1 format is running fine for a week.
> But the cluster on the 0.11.0.1 format is consistently having memory issues, 
> only on the two upgraded brokers running 1.0.0.
> The first occurrence of the error comes along with this stack trace
> {noformat}
> {"timestamp":"2017-11-06 
> 14:22:32,402","level":"ERROR","logger":"kafka.server.KafkaApis","thread":"kafka-request-handler-7","message":"[KafkaApi-1]
>  Error when handling request 
> 

[jira] [Resolved] (KAFKA-6250) Kafka Connect requires permission to create internal topics even if they exist

2018-01-11 Thread Jason Gustafson (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6250?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jason Gustafson resolved KAFKA-6250.

Resolution: Fixed
  Assignee: Gavrie Philipson

> Kafka Connect requires permission to create internal topics even if they exist
> --
>
> Key: KAFKA-6250
> URL: https://issues.apache.org/jira/browse/KAFKA-6250
> Project: Kafka
>  Issue Type: Bug
>  Components: KafkaConnect
>Affects Versions: 0.11.0.1, 1.0.0
>Reporter: Gavrie Philipson
>Assignee: Gavrie Philipson
> Fix For: 1.1.0
>
>
> When using Kafka Connect with a cluster that doesn't allow the user to create 
> topics (due to ACL configuration), Connect fails when trying to create its 
> internal topics, even if these topics already exist.
> This happens specifically when using hosted [Aiven 
> Kafka|https://aiven.io/kafka], which does not permit creation of topics via 
> the Kafka Admin Client API.
> The problem is that Connect tries to create the topics, and ignores some 
> specific errors such as topics that already exist, but not authorization 
> errors.
> This is what happens:
> {noformat}
> 2017-11-21 15:57:24,176 [DistributedHerder] ERROR DistributedHerder:206 - 
> Uncaught exception in herder work thread, exiting:
> org.apache.kafka.connect.errors.ConnectException: Error while attempting to 
> create/find topic(s) 'connect-offsets'
>   at 
> org.apache.kafka.connect.util.TopicAdmin.createTopics(TopicAdmin.java:245)
>   at 
> org.apache.kafka.connect.storage.KafkaOffsetBackingStore$1.run(KafkaOffsetBackingStore.java:99)
>   at 
> org.apache.kafka.connect.util.KafkaBasedLog.start(KafkaBasedLog.java:126)
>   at 
> org.apache.kafka.connect.storage.KafkaOffsetBackingStore.start(KafkaOffsetBackingStore.java:109)
>   at org.apache.kafka.connect.runtime.Worker.start(Worker.java:146)
>   at 
> org.apache.kafka.connect.runtime.AbstractHerder.startServices(AbstractHerder.java:99)
>   at 
> org.apache.kafka.connect.runtime.distributed.DistributedHerder.run(DistributedHerder.java:194)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>   at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>   at java.lang.Thread.run(Thread.java:745)
> Caused by: java.util.concurrent.ExecutionException: 
> org.apache.kafka.common.errors.ClusterAuthorizationException: Cluster 
> authorization failed.
>   at 
> org.apache.kafka.common.internals.KafkaFutureImpl.wrapAndThrow(KafkaFutureImpl.java:45)
>   at 
> org.apache.kafka.common.internals.KafkaFutureImpl.access$000(KafkaFutureImpl.java:32)
>   at 
> org.apache.kafka.common.internals.KafkaFutureImpl$SingleWaiter.await(KafkaFutureImpl.java:89)
>   at 
> org.apache.kafka.common.internals.KafkaFutureImpl.get(KafkaFutureImpl.java:213)
>   at 
> org.apache.kafka.connect.util.TopicAdmin.createTopics(TopicAdmin.java:226)
>   ... 11 more
> Caused by: org.apache.kafka.common.errors.ClusterAuthorizationException: 
> Cluster authorization failed.
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-6250) Kafka Connect requires permission to create internal topics even if they exist

2018-01-11 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-6250?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16323245#comment-16323245
 ] 

ASF GitHub Bot commented on KAFKA-6250:
---

hachikuji closed pull request #4247: KAFKA-6250: Use existing Kafka Connect 
internal topics without requiring ACL
URL: https://github.com/apache/kafka/pull/4247
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java 
b/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java
index 5da4f2d00d0..ad21561baf2 100644
--- 
a/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java
+++ 
b/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java
@@ -21,6 +21,7 @@
 import org.apache.kafka.clients.admin.CreateTopicsOptions;
 import org.apache.kafka.clients.admin.NewTopic;
 import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
 import org.apache.kafka.common.errors.TimeoutException;
 import org.apache.kafka.common.errors.TopicExistsException;
 import org.apache.kafka.common.errors.UnsupportedVersionException;
@@ -229,13 +230,20 @@ public boolean createTopic(NewTopic topic) {
 newlyCreatedTopicNames.add(topic);
 } catch (ExecutionException e) {
 Throwable cause = e.getCause();
-if (e.getCause() instanceof TopicExistsException) {
+if (cause instanceof TopicExistsException) {
 log.debug("Found existing topic '{}' on the brokers at 
{}", topic, bootstrapServers);
 continue;
 }
 if (cause instanceof UnsupportedVersionException) {
-log.debug("Unable to use Kafka admin client to create 
topic descriptions for '{}' using the brokers at {}," +
-  "falling back to assume topic(s) exist 
or will be auto-created by the broker", topicNameList, bootstrapServers);
+log.debug("Unable to create topic(s) '{}' since the 
brokers at {} do not support the CreateTopics API.",
+" Falling back to assume topic(s) exist or will be 
auto-created by the broker.",
+topicNameList, bootstrapServers);
+return Collections.emptySet();
+}
+if (cause instanceof ClusterAuthorizationException) {
+log.debug("Not authorized to create topic(s) '{}'." +
+" Falling back to assume topic(s) exist or will be 
auto-created by the broker.",
+topicNameList, bootstrapServers);
 return Collections.emptySet();
 }
 if (cause instanceof TimeoutException) {
diff --git 
a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java
 
b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java
index c58d6741f58..cda68795689 100644
--- 
a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java
+++ 
b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java
@@ -60,6 +60,19 @@ public void returnNullWithApiVersionMismatch() {
 }
 }
 
+@Test
+public void returnNullWithClusterAuthorizationFailure() {
+final NewTopic newTopic = 
TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build();
+Cluster cluster = createCluster(1);
+try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(cluster)) 
{
+env.kafkaClient().prepareMetadataUpdate(env.cluster(), 
Collections.emptySet());
+
env.kafkaClient().prepareResponse(createTopicResponseWithClusterAuthorizationException(newTopic));
+TopicAdmin admin = new TopicAdmin(null, env.adminClient());
+boolean created = admin.createTopic(newTopic);
+assertFalse(created);
+}
+}
+
 @Test
 public void shouldNotCreateTopicWhenItAlreadyExists() {
 NewTopic newTopic = 
TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build();
@@ -120,6 +133,10 @@ private CreateTopicsResponse 
createTopicResponseWithUnsupportedVersion(NewTopic.
 return createTopicResponse(new ApiError(Errors.UNSUPPORTED_VERSION, 
"This version of the API is not supported"), topics);
 }
 
+private CreateTopicsResponse 
createTopicResponseWithClusterAuthorizationException(NewTopic... topics) {
+return createTopicResponse(new 
ApiError(Errors.CLUSTER_AUTHORIZATION_FAILED, "Not authorized to create 
topic(s)"), topics);
+ 

[jira] [Commented] (KAFKA-4850) RocksDb cannot use Bloom Filters

2018-01-11 Thread Peter Davis (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16323184#comment-16323184
 ] 

Peter Davis commented on KAFKA-4850:


Request: RocksDB memory usage (off Java heap) is already hard to fathom and can 
be substantial, and [the 
documentation|https://github.com/facebook/rocksdb/wiki/RocksDB-Bloom-Filter#memory-usage]
 is anything but clear.  If adding bloom filters affects memory usage, can the 
impact please be documented?

> RocksDb cannot use Bloom Filters
> 
>
> Key: KAFKA-4850
> URL: https://issues.apache.org/jira/browse/KAFKA-4850
> Project: Kafka
>  Issue Type: Improvement
>  Components: streams
>Affects Versions: 0.10.2.0
>Reporter: Eno Thereska
>Assignee: Bharat Viswanadham
> Fix For: 1.1.0
>
>
> Bloom Filters would speed up RocksDb lookups. However they currently do not 
> work in RocksDb 5.0.2. This has been fixed in trunk, but we'll have to wait 
> until that is released and tested. 
> Then we can add the line in RocksDbStore.java in openDb:
> tableConfig.setFilter(new BloomFilter(10));



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-6441) FetchRequest populates buffer of size MinBytes, even if response is smaller

2018-01-11 Thread Ivan Babrou (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-6441?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16323073#comment-16323073
 ] 

Ivan Babrou commented on KAFKA-6441:


I think it's a bit different, buffers for each partition are allocated based on 
maxBytes:

{noformat}
2018/01/11 21:48:58 Request: max wait time = 500, min bytes = 1, max bytes = 
104857600, isolation = 0, num blocks = 1
2018/01/11 21:48:58   fetch request block for partition 0: 
{fetchOffset:7075063209, maxBytes:2097152}
2018/01/11 21:48:58 Request: max wait time = 500, min bytes = 1, max bytes = 
104857600, isolation = 0, num blocks = 1
2018/01/11 21:48:58   fetch request block for partition 0: 
{fetchOffset:7075063209, maxBytes:2097152}
{noformat}

Here fetchRequestBlock translates to roughly to  PartitionData(offset, 
logStartOffset, maxBytes)

if I dump individual segments from the log, I see individual messages:

{noformat}
baseOffset: 15165279076 lastOffset: 15165279076 baseSequence: -1 lastSequence: 
-1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 140 isTransactional: 
false position: 9241092 CreateTime: 1515699408944 isvalid: true size: 910 
magic: 2 compresscodec: NONE crc: 456596511
baseOffset: 15165279077 lastOffset: 15165279077 baseSequence: -1 lastSequence: 
-1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 140 isTransactional: 
false position: 9242002 CreateTime: 1515699408955 isvalid: true size: 910 
magic: 2 compresscodec: NONE crc: 465015653
baseOffset: 15165279078 lastOffset: 15165279078 baseSequence: -1 lastSequence: 
-1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 140 isTransactional: 
false position: 9242912 CreateTime: 1515699408960 isvalid: true size: 908 
magic: 2 compresscodec: NONE crc: 1569816164
baseOffset: 15165279079 lastOffset: 15165279079 baseSequence: -1 lastSequence: 
-1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 140 isTransactional: 
false position: 9243820 CreateTime: 1515699408997 isvalid: true size: 915 
magic: 2 compresscodec: NONE crc: 1894915965
baseOffset: 15165279080 lastOffset: 15165279080 baseSequence: -1 lastSequence: 
-1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 140 isTransactional: 
false position: 9244735 CreateTime: 1515699409010 isvalid: true size: 916 
magic: 2 compresscodec: NONE crc: 2124364233
baseOffset: 15165279081 lastOffset: 15165279081 baseSequence: -1 lastSequence: 
-1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 140 isTransactional: 
false position: 9245651 CreateTime: 1515699409035 isvalid: true size: 918 
magic: 2 compresscodec: NONE crc: 1889246530
baseOffset: 15165279082 lastOffset: 15165279082 baseSequence: -1 lastSequence: 
-1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 140 isTransactional: 
false position: 9246569 CreateTime: 1515699409038 isvalid: true size: 914 
magic: 2 compresscodec: NONE crc: 877751927
baseOffset: 15165279083 lastOffset: 15165279083 baseSequence: -1 lastSequence: 
-1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 140 isTransactional: 
false position: 9247483 CreateTime: 1515699409061 isvalid: true size: 915 
magic: 2 compresscodec: NONE crc: 3313577153
baseOffset: 15165279084 lastOffset: 15165279084 baseSequence: -1 lastSequence: 
-1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 140 isTransactional: 
false position: 9248398 CreateTime: 1515699409132 isvalid: true size: 912 
magic: 2 compresscodec: NONE crc: 1951840175
baseOffset: 15165279085 lastOffset: 15165279085 baseSequence: -1 lastSequence: 
-1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 140 isTransactional: 
false position: 9249310 CreateTime: 1515699409133 isvalid: true size: 915 
magic: 2 compresscodec: NONE crc: 1357735233
baseOffset: 15165279086 lastOffset: 15165279086 baseSequence: -1 lastSequence: 
-1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 140 isTransactional: 
false position: 9250225 CreateTime: 1515699409137 isvalid: true size: 920 
magic: 2 compresscodec: NONE crc: 899719626
baseOffset: 15165279087 lastOffset: 15165279087 baseSequence: -1 lastSequence: 
-1 producerId: -1 producerEpoch: -1 partitionLeaderEpoch: 140 isTransactional: 
false position: 9251145 CreateTime: 1515699409162 isvalid: true size: 915 
magic: 2 compresscodec: NONE crc: 1993963751
{noformat}

These should be combined when returned to consumer if buffer is large enough, 
but they are not for some reason.

> FetchRequest populates buffer of size MinBytes, even if response is smaller
> ---
>
> Key: KAFKA-6441
> URL: https://issues.apache.org/jira/browse/KAFKA-6441
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.11.0.1
>Reporter: Ivan Babrou
>
> We're using Sarama Go client as consumer, but I don't think it's relevant. 
> Producer is syslog-ng with Kafka output, I'm not quite sure which log 

[jira] [Commented] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Jan Filipiak (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16323021#comment-16323021
 ] 

Jan Filipiak commented on KAFKA-6442:
-

I guess discussing this on the mailing-list / slack first could help quicker.

Did you set "unclean.leader.election.enable" to false? An unclean election 
should allow broker 1 to take leadership with whatever it has.

If unclean election is true and its still not stepping up I usually delete the 
/controller node in zk to have a new controller elected to take care of these 
partitions again.


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: Andreas
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> Topic: topicA Partition: 32 Leader: -1 Replicas: 1,6,2,7,3,8 Isr: 
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from which I cannot recover because I cannot run 
> {{kafka-reassign-partitions.sh}}.
> Is there a way to recover from such a situation? 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-5413) Log cleaner fails due to large offset in segment file

2018-01-11 Thread Andrew Olson (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-5413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16322857#comment-16322857
 ] 

Andrew Olson commented on KAFKA-5413:
-

Thanks for the insight, we'll keep a watchful eye on it.

> Log cleaner fails due to large offset in segment file
> -
>
> Key: KAFKA-5413
> URL: https://issues.apache.org/jira/browse/KAFKA-5413
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.10.2.0, 0.10.2.1
> Environment: Ubuntu 14.04 LTS, Oracle Java 8u92, kafka_2.11-0.10.2.0
>Reporter: Nicholas Ngorok
>Assignee: Kelvin Rutt
>Priority: Critical
>  Labels: reliability
> Fix For: 0.10.2.2, 0.11.0.0
>
> Attachments: .index.cleaned, 
> .log, .log.cleaned, 
> .timeindex.cleaned, 002147422683.log, 
> kafka-5413.patch
>
>
> The log cleaner thread in our brokers is failing with the trace below
> {noformat}
> [2017-06-08 15:49:54,822] INFO {kafka-log-cleaner-thread-0} Cleaner 0: 
> Cleaning segment 0 in log __consumer_offsets-12 (largest timestamp Thu Jun 08 
> 15:48:59 PDT 2017) into 0, retaining deletes. (kafka.log.LogCleaner)
> [2017-06-08 15:49:54,822] INFO {kafka-log-cleaner-thread-0} Cleaner 0: 
> Cleaning segment 2147343575 in log __consumer_offsets-12 (largest timestamp 
> Thu Jun 08 15:49:06 PDT 2017) into 0, retaining deletes. 
> (kafka.log.LogCleaner)
> [2017-06-08 15:49:54,834] ERROR {kafka-log-cleaner-thread-0} 
> [kafka-log-cleaner-thread-0], Error due to  (kafka.log.LogCleaner)
> java.lang.IllegalArgumentException: requirement failed: largest offset in 
> message set can not be safely converted to relative offset.
> at scala.Predef$.require(Predef.scala:224)
> at kafka.log.LogSegment.append(LogSegment.scala:109)
> at kafka.log.Cleaner.cleanInto(LogCleaner.scala:478)
> at 
> kafka.log.Cleaner$$anonfun$cleanSegments$1.apply(LogCleaner.scala:405)
> at 
> kafka.log.Cleaner$$anonfun$cleanSegments$1.apply(LogCleaner.scala:401)
> at scala.collection.immutable.List.foreach(List.scala:381)
> at kafka.log.Cleaner.cleanSegments(LogCleaner.scala:401)
> at kafka.log.Cleaner$$anonfun$clean$4.apply(LogCleaner.scala:363)
> at kafka.log.Cleaner$$anonfun$clean$4.apply(LogCleaner.scala:362)
> at scala.collection.immutable.List.foreach(List.scala:381)
> at kafka.log.Cleaner.clean(LogCleaner.scala:362)
> at 
> kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:241)
> at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:220)
> at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)
> [2017-06-08 15:49:54,835] INFO {kafka-log-cleaner-thread-0} 
> [kafka-log-cleaner-thread-0], Stopped  (kafka.log.LogCleaner)
> {noformat}
> This seems to point at the specific line [here| 
> https://github.com/apache/kafka/blob/0.11.0/core/src/main/scala/kafka/log/LogSegment.scala#L92]
>  in the kafka src where the difference is actually larger than MAXINT as both 
> baseOffset and offset are of type long. It was introduced in this [pr| 
> https://github.com/apache/kafka/pull/2210/files/56d1f8196b77a47b176b7bbd1e4220a3be827631]
> These were the outputs of dumping the first two log segments
> {noformat}
> :~$ /usr/bin/kafka-run-class kafka.tools.DumpLogSegments --deep-iteration 
> --files /kafka-logs/__consumer_offsets-12/000
> 0.log
> Dumping /kafka-logs/__consumer_offsets-12/.log
> Starting offset: 0
> offset: 1810054758 position: 0 NoTimestampType: -1 isvalid: true payloadsize: 
> -1 magic: 0 compresscodec: NONE crc: 3127861909 keysize: 34
> :~$ /usr/bin/kafka-run-class kafka.tools.DumpLogSegments --deep-iteration 
> --files /kafka-logs/__consumer_offsets-12/000
> 0002147343575.log
> Dumping /kafka-logs/__consumer_offsets-12/002147343575.log
> Starting offset: 2147343575
> offset: 2147539884 position: 0 NoTimestampType: -1 isvalid: true paylo
> adsize: -1 magic: 0 compresscodec: NONE crc: 2282192097 keysize: 34
> {noformat}
> My guess is that since 2147539884 is larger than MAXINT, we are hitting this 
> exception. Was there a specific reason, this check was added in 0.10.2?
> E.g. if the first offset is a key = "key 0" and then we have MAXINT + 1 of 
> "key 1" following, wouldn't we run into this situation whenever the log 
> cleaner runs?



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-5413) Log cleaner fails due to large offset in segment file

2018-01-11 Thread Philippe Laflamme (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-5413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16322832#comment-16322832
 ] 

Philippe Laflamme commented on KAFKA-5413:
--

Yes, that's what happens when the log cleaner dies for any reason (the 
compacted logs are no longer compacted and will grow unbounded). The original 
issue has no workaround, meaning that the log cleaner will die again after 
restarting the broker...

FWIW, we ended up upgrading to 0.11.x since there was clearly no intention on 
releasing 0.10.2.2.

> Log cleaner fails due to large offset in segment file
> -
>
> Key: KAFKA-5413
> URL: https://issues.apache.org/jira/browse/KAFKA-5413
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.10.2.0, 0.10.2.1
> Environment: Ubuntu 14.04 LTS, Oracle Java 8u92, kafka_2.11-0.10.2.0
>Reporter: Nicholas Ngorok
>Assignee: Kelvin Rutt
>Priority: Critical
>  Labels: reliability
> Fix For: 0.10.2.2, 0.11.0.0
>
> Attachments: .index.cleaned, 
> .log, .log.cleaned, 
> .timeindex.cleaned, 002147422683.log, 
> kafka-5413.patch
>
>
> The log cleaner thread in our brokers is failing with the trace below
> {noformat}
> [2017-06-08 15:49:54,822] INFO {kafka-log-cleaner-thread-0} Cleaner 0: 
> Cleaning segment 0 in log __consumer_offsets-12 (largest timestamp Thu Jun 08 
> 15:48:59 PDT 2017) into 0, retaining deletes. (kafka.log.LogCleaner)
> [2017-06-08 15:49:54,822] INFO {kafka-log-cleaner-thread-0} Cleaner 0: 
> Cleaning segment 2147343575 in log __consumer_offsets-12 (largest timestamp 
> Thu Jun 08 15:49:06 PDT 2017) into 0, retaining deletes. 
> (kafka.log.LogCleaner)
> [2017-06-08 15:49:54,834] ERROR {kafka-log-cleaner-thread-0} 
> [kafka-log-cleaner-thread-0], Error due to  (kafka.log.LogCleaner)
> java.lang.IllegalArgumentException: requirement failed: largest offset in 
> message set can not be safely converted to relative offset.
> at scala.Predef$.require(Predef.scala:224)
> at kafka.log.LogSegment.append(LogSegment.scala:109)
> at kafka.log.Cleaner.cleanInto(LogCleaner.scala:478)
> at 
> kafka.log.Cleaner$$anonfun$cleanSegments$1.apply(LogCleaner.scala:405)
> at 
> kafka.log.Cleaner$$anonfun$cleanSegments$1.apply(LogCleaner.scala:401)
> at scala.collection.immutable.List.foreach(List.scala:381)
> at kafka.log.Cleaner.cleanSegments(LogCleaner.scala:401)
> at kafka.log.Cleaner$$anonfun$clean$4.apply(LogCleaner.scala:363)
> at kafka.log.Cleaner$$anonfun$clean$4.apply(LogCleaner.scala:362)
> at scala.collection.immutable.List.foreach(List.scala:381)
> at kafka.log.Cleaner.clean(LogCleaner.scala:362)
> at 
> kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:241)
> at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:220)
> at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)
> [2017-06-08 15:49:54,835] INFO {kafka-log-cleaner-thread-0} 
> [kafka-log-cleaner-thread-0], Stopped  (kafka.log.LogCleaner)
> {noformat}
> This seems to point at the specific line [here| 
> https://github.com/apache/kafka/blob/0.11.0/core/src/main/scala/kafka/log/LogSegment.scala#L92]
>  in the kafka src where the difference is actually larger than MAXINT as both 
> baseOffset and offset are of type long. It was introduced in this [pr| 
> https://github.com/apache/kafka/pull/2210/files/56d1f8196b77a47b176b7bbd1e4220a3be827631]
> These were the outputs of dumping the first two log segments
> {noformat}
> :~$ /usr/bin/kafka-run-class kafka.tools.DumpLogSegments --deep-iteration 
> --files /kafka-logs/__consumer_offsets-12/000
> 0.log
> Dumping /kafka-logs/__consumer_offsets-12/.log
> Starting offset: 0
> offset: 1810054758 position: 0 NoTimestampType: -1 isvalid: true payloadsize: 
> -1 magic: 0 compresscodec: NONE crc: 3127861909 keysize: 34
> :~$ /usr/bin/kafka-run-class kafka.tools.DumpLogSegments --deep-iteration 
> --files /kafka-logs/__consumer_offsets-12/000
> 0002147343575.log
> Dumping /kafka-logs/__consumer_offsets-12/002147343575.log
> Starting offset: 2147343575
> offset: 2147539884 position: 0 NoTimestampType: -1 isvalid: true paylo
> adsize: -1 magic: 0 compresscodec: NONE crc: 2282192097 keysize: 34
> {noformat}
> My guess is that since 2147539884 is larger than MAXINT, we are hitting this 
> exception. Was there a specific reason, this check was added in 0.10.2?
> E.g. if the first offset is a key = "key 0" and then we have MAXINT + 1 of 
> "key 1" following, wouldn't we run into this situation whenever the log 
> cleaner runs?



--
This message was 

[jira] [Commented] (KAFKA-6443) KTable involved in multiple joins could result in duplicate results

2018-01-11 Thread Guozhang Wang (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-6443?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16322822#comment-16322822
 ] 

Guozhang Wang commented on KAFKA-6443:
--

I think whether it should be considered a bug or we'd say it is acceptable 
result is still open for discussion.

> KTable involved in multiple joins could result in duplicate results
> ---
>
> Key: KAFKA-6443
> URL: https://issues.apache.org/jira/browse/KAFKA-6443
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Reporter: Guozhang Wang
>
> Consider the following multi table-table joins:
> {code}
> table1.join(table2).join(table2);// "join" could be replaced with 
> "leftJoin" and "outerJoin"
> {code}
> where {{table2}} is involved multiple times in this multi-way joins. In this 
> case, when a new record from the source topic of {{table2}} is being 
> processing, it will send to two children down in the topology and hence may 
> resulting in duplicated join results depending on the join types.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-6205) Have State Stores Restore Before Initializing Toplogy

2018-01-11 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-6205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16322806#comment-16322806
 ] 

ASF GitHub Bot commented on KAFKA-6205:
---

bbejeck opened a new pull request #4415: KAFKA-6205: initialize topology after 
state stores restoration completed
URL: https://github.com/apache/kafka/pull/4415
 
 
   Initialize topology after state store restoration.  
   Although IMHO updating some of the existing tests demonstrates the correct 
order of operations, I'll probably add an integration test, but I wanted to get 
this PR in for feedback on the approach.
   
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage, and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Have State Stores Restore Before Initializing Toplogy
> -
>
> Key: KAFKA-6205
> URL: https://issues.apache.org/jira/browse/KAFKA-6205
> Project: Kafka
>  Issue Type: Improvement
>  Components: streams
>Affects Versions: 1.0.0, 0.11.0.2
>Reporter: Bill Bejeck
>Assignee: Bill Bejeck
> Fix For: 1.0.1, 0.11.0.3
>
>
> Streams should restore state stores (if needed) before initializing the 
> topology.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-6443) KTable involved in multiple joins could result in duplicate results

2018-01-11 Thread Matthias J. Sax (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-6443?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16322794#comment-16322794
 ] 

Matthias J. Sax commented on KAFKA-6443:


I agree, that we would get two update records, but they won't be equal? The 
first should be an "incorrect" intermediate update, while the second would be 
the correct one. Or do I miss something?

> KTable involved in multiple joins could result in duplicate results
> ---
>
> Key: KAFKA-6443
> URL: https://issues.apache.org/jira/browse/KAFKA-6443
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Reporter: Guozhang Wang
>
> Consider the following multi table-table joins:
> {code}
> table1.join(table2).join(table2);// "join" could be replaced with 
> "leftJoin" and "outerJoin"
> {code}
> where {{table2}} is involved multiple times in this multi-way joins. In this 
> case, when a new record from the source topic of {{table2}} is being 
> processing, it will send to two children down in the topology and hence may 
> resulting in duplicated join results depending on the join types.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (KAFKA-6443) KTable involved in multiple joins could result in duplicate results

2018-01-11 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-6443:


 Summary: KTable involved in multiple joins could result in 
duplicate results
 Key: KAFKA-6443
 URL: https://issues.apache.org/jira/browse/KAFKA-6443
 Project: Kafka
  Issue Type: Bug
  Components: streams
Reporter: Guozhang Wang


Consider the following multi table-table joins:

{code}
table1.join(table2).join(table2);// "join" could be replaced with 
"leftJoin" and "outerJoin"
{code}

where {{table2}} is involved multiple times in this multi-way joins. In this 
case, when a new record from the source topic of {{table2}} is being 
processing, it will send to two children down in the topology and hence may 
resulting in duplicated join results depending on the join types.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Comment Edited] (KAFKA-5413) Log cleaner fails due to large offset in segment file

2018-01-11 Thread Andrew Olson (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-5413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16322759#comment-16322759
 ] 

Andrew Olson edited comment on KAFKA-5413 at 1/11/18 7:06 PM:
--

We ran into this for a partition of the __consumer_offsets topic. The issue was 
discovered when we noticed that the open file count for a Kafka broker had been 
steadily growing for a couple months, and was about 2x higher than any other 
broker in the cluster. When we restarted this broker it seemed able to recover, 
deleting a large number of old log segments, with the open file count returning 
to a more normal value.


was (Author: noslowerdna):
We ran into this for a partition of the __consumer_offsets topic. The issue was 
discovered when we noticed that the open file count for a Kafka broker had been 
steadily growing for a couple months, and was about 2x higher than any other 
broker in the cluster. When we restarted this broker it seemed able to recover, 
deleting a large number of old segments for __consumer_offsets, with the open 
file count returning to a more normal value.

> Log cleaner fails due to large offset in segment file
> -
>
> Key: KAFKA-5413
> URL: https://issues.apache.org/jira/browse/KAFKA-5413
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.10.2.0, 0.10.2.1
> Environment: Ubuntu 14.04 LTS, Oracle Java 8u92, kafka_2.11-0.10.2.0
>Reporter: Nicholas Ngorok
>Assignee: Kelvin Rutt
>Priority: Critical
>  Labels: reliability
> Fix For: 0.10.2.2, 0.11.0.0
>
> Attachments: .index.cleaned, 
> .log, .log.cleaned, 
> .timeindex.cleaned, 002147422683.log, 
> kafka-5413.patch
>
>
> The log cleaner thread in our brokers is failing with the trace below
> {noformat}
> [2017-06-08 15:49:54,822] INFO {kafka-log-cleaner-thread-0} Cleaner 0: 
> Cleaning segment 0 in log __consumer_offsets-12 (largest timestamp Thu Jun 08 
> 15:48:59 PDT 2017) into 0, retaining deletes. (kafka.log.LogCleaner)
> [2017-06-08 15:49:54,822] INFO {kafka-log-cleaner-thread-0} Cleaner 0: 
> Cleaning segment 2147343575 in log __consumer_offsets-12 (largest timestamp 
> Thu Jun 08 15:49:06 PDT 2017) into 0, retaining deletes. 
> (kafka.log.LogCleaner)
> [2017-06-08 15:49:54,834] ERROR {kafka-log-cleaner-thread-0} 
> [kafka-log-cleaner-thread-0], Error due to  (kafka.log.LogCleaner)
> java.lang.IllegalArgumentException: requirement failed: largest offset in 
> message set can not be safely converted to relative offset.
> at scala.Predef$.require(Predef.scala:224)
> at kafka.log.LogSegment.append(LogSegment.scala:109)
> at kafka.log.Cleaner.cleanInto(LogCleaner.scala:478)
> at 
> kafka.log.Cleaner$$anonfun$cleanSegments$1.apply(LogCleaner.scala:405)
> at 
> kafka.log.Cleaner$$anonfun$cleanSegments$1.apply(LogCleaner.scala:401)
> at scala.collection.immutable.List.foreach(List.scala:381)
> at kafka.log.Cleaner.cleanSegments(LogCleaner.scala:401)
> at kafka.log.Cleaner$$anonfun$clean$4.apply(LogCleaner.scala:363)
> at kafka.log.Cleaner$$anonfun$clean$4.apply(LogCleaner.scala:362)
> at scala.collection.immutable.List.foreach(List.scala:381)
> at kafka.log.Cleaner.clean(LogCleaner.scala:362)
> at 
> kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:241)
> at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:220)
> at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)
> [2017-06-08 15:49:54,835] INFO {kafka-log-cleaner-thread-0} 
> [kafka-log-cleaner-thread-0], Stopped  (kafka.log.LogCleaner)
> {noformat}
> This seems to point at the specific line [here| 
> https://github.com/apache/kafka/blob/0.11.0/core/src/main/scala/kafka/log/LogSegment.scala#L92]
>  in the kafka src where the difference is actually larger than MAXINT as both 
> baseOffset and offset are of type long. It was introduced in this [pr| 
> https://github.com/apache/kafka/pull/2210/files/56d1f8196b77a47b176b7bbd1e4220a3be827631]
> These were the outputs of dumping the first two log segments
> {noformat}
> :~$ /usr/bin/kafka-run-class kafka.tools.DumpLogSegments --deep-iteration 
> --files /kafka-logs/__consumer_offsets-12/000
> 0.log
> Dumping /kafka-logs/__consumer_offsets-12/.log
> Starting offset: 0
> offset: 1810054758 position: 0 NoTimestampType: -1 isvalid: true payloadsize: 
> -1 magic: 0 compresscodec: NONE crc: 3127861909 keysize: 34
> :~$ /usr/bin/kafka-run-class kafka.tools.DumpLogSegments --deep-iteration 
> --files /kafka-logs/__consumer_offsets-12/000
> 0002147343575.log
> Dumping 

[jira] [Commented] (KAFKA-5413) Log cleaner fails due to large offset in segment file

2018-01-11 Thread Andrew Olson (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-5413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16322759#comment-16322759
 ] 

Andrew Olson commented on KAFKA-5413:
-

We ran into this for a partition of the __consumer_offsets topic. The issue was 
discovered when we noticed that the open file count for a Kafka broker had been 
steadily growing for a couple months, and was about 2x higher than any other 
broker in the cluster. When we restarted this broker it seemed able to recover, 
deleting a large number of old segments for __consumer_offsets, with the open 
file count returning to a more normal value.

> Log cleaner fails due to large offset in segment file
> -
>
> Key: KAFKA-5413
> URL: https://issues.apache.org/jira/browse/KAFKA-5413
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.10.2.0, 0.10.2.1
> Environment: Ubuntu 14.04 LTS, Oracle Java 8u92, kafka_2.11-0.10.2.0
>Reporter: Nicholas Ngorok
>Assignee: Kelvin Rutt
>Priority: Critical
>  Labels: reliability
> Fix For: 0.10.2.2, 0.11.0.0
>
> Attachments: .index.cleaned, 
> .log, .log.cleaned, 
> .timeindex.cleaned, 002147422683.log, 
> kafka-5413.patch
>
>
> The log cleaner thread in our brokers is failing with the trace below
> {noformat}
> [2017-06-08 15:49:54,822] INFO {kafka-log-cleaner-thread-0} Cleaner 0: 
> Cleaning segment 0 in log __consumer_offsets-12 (largest timestamp Thu Jun 08 
> 15:48:59 PDT 2017) into 0, retaining deletes. (kafka.log.LogCleaner)
> [2017-06-08 15:49:54,822] INFO {kafka-log-cleaner-thread-0} Cleaner 0: 
> Cleaning segment 2147343575 in log __consumer_offsets-12 (largest timestamp 
> Thu Jun 08 15:49:06 PDT 2017) into 0, retaining deletes. 
> (kafka.log.LogCleaner)
> [2017-06-08 15:49:54,834] ERROR {kafka-log-cleaner-thread-0} 
> [kafka-log-cleaner-thread-0], Error due to  (kafka.log.LogCleaner)
> java.lang.IllegalArgumentException: requirement failed: largest offset in 
> message set can not be safely converted to relative offset.
> at scala.Predef$.require(Predef.scala:224)
> at kafka.log.LogSegment.append(LogSegment.scala:109)
> at kafka.log.Cleaner.cleanInto(LogCleaner.scala:478)
> at 
> kafka.log.Cleaner$$anonfun$cleanSegments$1.apply(LogCleaner.scala:405)
> at 
> kafka.log.Cleaner$$anonfun$cleanSegments$1.apply(LogCleaner.scala:401)
> at scala.collection.immutable.List.foreach(List.scala:381)
> at kafka.log.Cleaner.cleanSegments(LogCleaner.scala:401)
> at kafka.log.Cleaner$$anonfun$clean$4.apply(LogCleaner.scala:363)
> at kafka.log.Cleaner$$anonfun$clean$4.apply(LogCleaner.scala:362)
> at scala.collection.immutable.List.foreach(List.scala:381)
> at kafka.log.Cleaner.clean(LogCleaner.scala:362)
> at 
> kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:241)
> at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:220)
> at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)
> [2017-06-08 15:49:54,835] INFO {kafka-log-cleaner-thread-0} 
> [kafka-log-cleaner-thread-0], Stopped  (kafka.log.LogCleaner)
> {noformat}
> This seems to point at the specific line [here| 
> https://github.com/apache/kafka/blob/0.11.0/core/src/main/scala/kafka/log/LogSegment.scala#L92]
>  in the kafka src where the difference is actually larger than MAXINT as both 
> baseOffset and offset are of type long. It was introduced in this [pr| 
> https://github.com/apache/kafka/pull/2210/files/56d1f8196b77a47b176b7bbd1e4220a3be827631]
> These were the outputs of dumping the first two log segments
> {noformat}
> :~$ /usr/bin/kafka-run-class kafka.tools.DumpLogSegments --deep-iteration 
> --files /kafka-logs/__consumer_offsets-12/000
> 0.log
> Dumping /kafka-logs/__consumer_offsets-12/.log
> Starting offset: 0
> offset: 1810054758 position: 0 NoTimestampType: -1 isvalid: true payloadsize: 
> -1 magic: 0 compresscodec: NONE crc: 3127861909 keysize: 34
> :~$ /usr/bin/kafka-run-class kafka.tools.DumpLogSegments --deep-iteration 
> --files /kafka-logs/__consumer_offsets-12/000
> 0002147343575.log
> Dumping /kafka-logs/__consumer_offsets-12/002147343575.log
> Starting offset: 2147343575
> offset: 2147539884 position: 0 NoTimestampType: -1 isvalid: true paylo
> adsize: -1 magic: 0 compresscodec: NONE crc: 2282192097 keysize: 34
> {noformat}
> My guess is that since 2147539884 is larger than MAXINT, we are hitting this 
> exception. Was there a specific reason, this check was added in 0.10.2?
> E.g. if the first offset is a key = "key 0" and then we have MAXINT + 1 of 
> "key 1" following, wouldn't we 

[jira] [Commented] (KAFKA-6378) NullPointerException on KStream-GlobalKTable leftJoin when KeyValueMapper returns null

2018-01-11 Thread Matthias J. Sax (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-6378?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16322672#comment-16322672
 ] 

Matthias J. Sax commented on KAFKA-6378:


Cool. Thanks a lot!

> NullPointerException on KStream-GlobalKTable leftJoin when KeyValueMapper 
> returns null
> --
>
> Key: KAFKA-6378
> URL: https://issues.apache.org/jira/browse/KAFKA-6378
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 1.0.0
>Reporter: Andy Bryant
>
> On a Stream->GlobalKTable leftJoin if the KeyValueMapper returns null, the 
> stream fails with a NullPointerException (see stacktrace below). On Kafka 
> 0.11.0.0 the stream processes this successfully, calling the ValueJoiner with 
> the table value set to null.
> The use-case for this is joining a stream to a table containing reference 
> data where the stream foreign key may be null. There is no straight-forward 
> workaround in this case with Kafka 1.0.0 without having to resort to either 
> generating a key that will never match or branching the stream for records 
> that don't have the foreign key.
> Exception in thread "workshop-simple-example-client-StreamThread-1" 
> java.lang.NullPointerException
>   at java.base/java.util.Objects.requireNonNull(Objects.java:221)
>   at 
> org.apache.kafka.streams.state.internals.CachingKeyValueStore.get(CachingKeyValueStore.java:136)
>   at 
> org.apache.kafka.streams.state.internals.CachingKeyValueStore.get(CachingKeyValueStore.java:35)
>   at 
> org.apache.kafka.streams.state.internals.InnerMeteredKeyValueStore.get(InnerMeteredKeyValueStore.java:184)
>   at 
> org.apache.kafka.streams.state.internals.MeteredKeyValueBytesStore.get(MeteredKeyValueBytesStore.java:116)
>   at 
> org.apache.kafka.streams.kstream.internals.KTableSourceValueGetterSupplier$KTableSourceValueGetter.get(KTableSourceValueGetterSupplier.java:49)
>   at 
> org.apache.kafka.streams.kstream.internals.KStreamKTableJoinProcessor.process(KStreamKTableJoinProcessor.java:56)
>   at 
> org.apache.kafka.streams.processor.internals.ProcessorNode$1.run(ProcessorNode.java:46)
>   at 
> org.apache.kafka.streams.processor.internals.StreamsMetricsImpl.measureLatencyNs(StreamsMetricsImpl.java:208)
>   at 
> org.apache.kafka.streams.processor.internals.ProcessorNode.process(ProcessorNode.java:124)
>   at 
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:85)
>   at 
> org.apache.kafka.streams.processor.internals.SourceNode.process(SourceNode.java:80)
>   at 
> org.apache.kafka.streams.processor.internals.StreamTask.process(StreamTask.java:216)
>   at 
> org.apache.kafka.streams.processor.internals.AssignedTasks.process(AssignedTasks.java:403)
>   at 
> org.apache.kafka.streams.processor.internals.TaskManager.process(TaskManager.java:317)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.processAndMaybeCommit(StreamThread.java:942)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:822)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:774)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:744)



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-6265) GlobalKTable missing #queryableStoreName()

2018-01-11 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-6265?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16322667#comment-16322667
 ] 

ASF GitHub Bot commented on KAFKA-6265:
---

guozhangwang closed pull request #4413: [KAFKA-6265] GlobalKTable missing 
#queryableStoreName()
URL: https://github.com/apache/kafka/pull/4413
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html
index 43038118692..c68b9bf4333 100644
--- a/docs/streams/upgrade-guide.html
+++ b/docs/streams/upgrade-guide.html
@@ -71,6 +71,13 @@ Streams API
 to distinguish them from configurations of other clients that share 
the same config names.
 
 
+
+   New method in GlobalKTable
+
+
+A method has been provided such that it will return the store name 
associated with the GlobalKTable or null if the store 
name is non-queryable. 
+
+
 
 New methods in KafkaStreams:
 
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/GlobalKTable.java 
b/streams/src/main/java/org/apache/kafka/streams/kstream/GlobalKTable.java
index 72286c20529..e58f67fc5b3 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/GlobalKTable.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/GlobalKTable.java
@@ -67,4 +67,10 @@
  */
 @InterfaceStability.Evolving
 public interface GlobalKTable {
+/**
+ * Get the name of the local state store that can be used to query this 
{@code GlobalKTable}.
+ *
+ * @return the underlying state store name, or {@code null} if this {@code 
GlobalKTable} cannot be queried.
+ */
+String queryableStoreName();
 }
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GlobalKTableImpl.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GlobalKTableImpl.java
index 34e23752444..8fcdfed1e52 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GlobalKTableImpl.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GlobalKTableImpl.java
@@ -21,13 +21,29 @@
 public class GlobalKTableImpl implements GlobalKTable {
 
 private final KTableValueGetterSupplier valueGetterSupplier;
+private final boolean queryable;
 
 public GlobalKTableImpl(final KTableValueGetterSupplier 
valueGetterSupplier) {
 this.valueGetterSupplier = valueGetterSupplier;
+this.queryable = true;
+}
+
+public GlobalKTableImpl(final KTableValueGetterSupplier 
valueGetterSupplier, 
+final boolean queryable) {
+this.valueGetterSupplier = valueGetterSupplier;
+this.queryable = queryable;
 }
 
 KTableValueGetterSupplier valueGetterSupplier() {
 return valueGetterSupplier;
 }
 
+@Override
+public String queryableStoreName() {
+if (!queryable) {
+return null;
+}
+return valueGetterSupplier.storeNames()[0];
+}
+
 }
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java
index 4308e5d0c50..2a8a89e1f0e 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java
@@ -158,7 +158,7 @@ public InternalStreamsBuilder(final InternalTopologyBuilder 
internalTopologyBuil
topic,
processorName,
tableSource);
-return new GlobalKTableImpl<>(new KTableSourceValueGetterSupplier(storeBuilder.name()));
+return new GlobalKTableImpl<>(new KTableSourceValueGetterSupplier(storeBuilder.name()), materialized.isQueryable());
 }
 
 @Override
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java
index 156acadbedd..b9ba6089497 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java
@@ -156,6 +156,32 @@ public void 
shouldStillMaterializeSourceKTableIfMaterializedIsntQueryable() thro
 assertEquals("topic2", 
topology.storeToChangelogTopic().get(storeName));
 

[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Description: 
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

*Topic topicA  Partition 32  Leader 1 
Replicas 162738  Isr *
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 

  was:
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

Topic topicA  Partition 32  Leader 1 
Replicas 162738  Isr 
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: Andreas
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> *Topic topicA  Partition 32  Leader 1 
> Replicas 162738  Isr *
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from which I cannot recover because I cannot run 
> 

[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Description: 
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

Topic topicA  Partition 32  Leader 1 
Replicas 162738  Isr 
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 

  was:
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA  Partition 32  Leader 1 }}{{ 
Replicas 162738  Isr }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: Andreas
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> Topic topicA  Partition 32  Leader 1 
> Replicas 162738  Isr 
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from which I cannot recover because I cannot run 
> 

[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Description: 
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

Topic topicA  Partition 32  Leader 1 
Replicas 162738  Isr 
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 

  was:
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

*Topic topicA  Partition 32  Leader 1 
Replicas 162738  Isr *
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: Andreas
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> Topic topicA  Partition 32  Leader 1 
> Replicas 162738  Isr 
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from which I cannot recover because I cannot run 
> 

[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Description: 
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA  Partition 32  Leader 1 }}{{ 
Replicas 162738  Isr }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 

  was:
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA  Partition 32  Leader 1 }} 
Replicas 162738  Isr }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: Andreas
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> {{Topic topicA  Partition 32  Leader 1 }}{{ 
> Replicas 162738  Isr }}
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from which I cannot recover because I 

[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Description: 
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA  Partition 32  Leader 1 }} 
Replicas 162738  Isr }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 

  was:
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA  Partition 32  Leader 1  
Replicas 162738  Isr }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: Andreas
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> {{Topic topicA  Partition 32  Leader 1 }} 
> Replicas 162738  Isr }}
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from which I cannot recover because I cannot 

[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Description: 
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA  Partition 32  Leader 1  
Replicas 162738  Isr }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 

  was:
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA  Partition 32  Leader 1  
Replicas 1,6,2,7,3,8  Isr }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: Andreas
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> {{Topic topicA  Partition 32  Leader 1  
> Replicas 162738  Isr }}
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from which I cannot recover because I cannot 

[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Description: 
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA  Partition 32  Leader 1  
Replicas 1,6,2,7,3,8  Isr }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 

  was:
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA  Partition 32  Leader -1 }}  
Replicas 1,6,2,7,3,8  Isr }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: Andreas
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> {{Topic topicA  Partition 32  Leader 1  
> Replicas 1,6,2,7,3,8  Isr }}
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from which I cannot recover 

[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Description: 
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA  Partition 32  Leader -1 }}  
Replicas 1,6,2,7,3,8  Isr }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 

  was:
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA  Partition 32}}  Leader -1   
Replicas 1,6,2,7,3,8  Isr }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: Andreas
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> {{Topic topicA  Partition 32  Leader -1 }}  
> Replicas 1,6,2,7,3,8  Isr }}
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from which I cannot 

[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Description: 
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA  Partition 32  Leader -1   
Replicas 1,6,2,7,3,8  Isr }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 

  was:
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA }}   Partition: 32  Leader: -1   Replicas: 
1,6,2,7,3,8  Isr: }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: Andreas
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> {{Topic topicA  Partition 32  Leader -1   
> Replicas 1,6,2,7,3,8  Isr }}
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from which I cannot 

[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Description: 
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA  Partition 32}}  Leader -1   
Replicas 1,6,2,7,3,8  Isr }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 

  was:
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA  Partition 32  Leader -1   
Replicas 1,6,2,7,3,8  Isr }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: Andreas
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> {{Topic topicA  Partition 32}}  Leader -1   
> Replicas 1,6,2,7,3,8  Isr }}
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from which I cannot 

[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Description: 
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA }}   Partition: 32  Leader: -1   Replicas: 
1,6,2,7,3,8  Isr: }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 

  was:
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic}}: topicAPartition: 32  Leader: -1   Replicas: 
1,6,2,7,3,8  Isr: }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: Andreas
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> {{Topic topicA }}   Partition: 32  Leader: -1   Replicas: 
> 1,6,2,7,3,8  Isr: }}
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from 

[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Description: 
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA }}   Partition: 32  Leader: -1   Replicas: 
1,6,2,7,3,8  Isr: }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 

  was:
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic topicA }}   Partition: 32  Leader: -1   Replicas: 
1,6,2,7,3,8  Isr: }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: Andreas
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> {{Topic topicA }}   Partition: 32  Leader: -1   Replicas: 
> 1,6,2,7,3,8  Isr: }}
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from 

[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Fix Version/s: (was: 0.8.2.1)

> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: Andreas
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> {{Topic: topicAPartition: 32  Leader: -1   Replicas: 
> 1,6,2,7,3,8  Isr: }}
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from which I cannot recover because I cannot run 
> {{kafka-reassign-partitions.sh}}.
> Is there a way to recover from such a situation? 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Affects Version/s: 0.8.2.1

> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: Andreas
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> {{Topic: topicAPartition: 32  Leader: -1   Replicas: 
> 1,6,2,7,3,8  Isr: }}
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from which I cannot recover because I cannot run 
> {{kafka-reassign-partitions.sh}}.
> Is there a way to recover from such a situation? 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Description: 
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic}}: topicAPartition: 32  Leader: -1   Replicas: 
1,6,2,7,3,8  Isr: }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 

  was:
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic: topicAPartition: 32  Leader: -1   Replicas: 
1,6,2,7,3,8  Isr: }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8.2.1
>Reporter: Andreas
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> {{Topic}}: topicAPartition: 32  Leader: -1   Replicas: 
> 1,6,2,7,3,8  Isr: }}
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from 

[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Description: 
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic: topicAPartition: 32  Leader: -1   Replicas: 
1,6,2,7,3,8  Isr: }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 

  was:
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

-{{ Topic: topicAPartition: 32  Leader: -1   Replicas: 
1,6,2,7,3,8  Isr: }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Reporter: Andreas
> Fix For: 0.8.2.1
>
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> {{Topic: topicAPartition: 32  Leader: -1   Replicas: 
> 1,6,2,7,3,8  Isr: }}
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from 

[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Description: 
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{ Topic: topicAPartition: 32   Leader: -1  Replicas: 1,6,2,7,3,8   
Isr: }}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because I cannot run 
{{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 

  was:
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic: topicA Partition: 32   Leader: -1  Replicas: 1,6,2,7,3,8   Isr:}}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because {{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Reporter: Andreas
> Fix For: 0.8.2.1
>
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> {{ Topic: topicA  Partition: 32   Leader: -1  Replicas: 1,6,2,7,3,8   
> Isr: }}
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from which I cannot recover because I cannot run 

[jira] [Updated] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas updated KAFKA-6442:
---
Description: 
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
{{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

{{Topic: topicA Partition: 32   Leader: -1  Replicas: 1,6,2,7,3,8   Isr:}}
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error {{Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.}}, 
from which I cannot recover because {{kafka-reassign-partitions.sh}}.

Is there a way to recover from such a situation? 

  was:
PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
`kafka-reassign-partitions.sh` to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

`Topic: topicA  Partition: 32   Leader: -1  Replicas: 1,6,2,7,3,8   Isr:`
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun `kafka-reassign-partitions.sh` 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error `Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.`, 
from which I cannot recover because `kafka-reassign-partitions.sh`.

Is there a way to recover from such a situation? 


> Catch 22 with cluster rebalancing
> -
>
> Key: KAFKA-6442
> URL: https://issues.apache.org/jira/browse/KAFKA-6442
> Project: Kafka
>  Issue Type: Bug
>Reporter: Andreas
> Fix For: 0.8.2.1
>
>
> PS. I classified this as a bug because I think the cluster should not be 
> stuck in that situation, apologies if that is wrong.
> Hi,
> I found myself in a situation a bit difficult to explain so I will skip the 
> how I ended up in this situation, but here is the problem.
> Some of the brokers of my cluster are permanently gone. Consequently, I had 
> some partitions that now had offline leaders etc so, I used the 
> {{kafka-reassign-partitions.sh}} to rebalance my topics and for the most part 
> that worked ok. Where that did not work ok, was for partitions that had 
> leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
> through to what now looks like
> {{Topic: topicA   Partition: 32   Leader: -1  Replicas: 1,6,2,7,3,8   
> Isr:}}
> (1,2,3 are legit, 6,7,8 permanently gone)
> So the first catch 22, is that I cannot elect a new leader, because the 
> leader needs to be elected from the ISR, and I cannot recreate the ISR 
> because the topic has no leader.
> The second catch 22 is that I cannot rerun {{kafka-reassign-partitions.sh}} 
> because the previous one is supposedly still in progress, and I cannot 
> increase the number of partitions to account for the now permanently offline 
> partitions, because that produces the following error {{Error while executing 
> topic command requirement failed: All partitions should have the same number 
> of replicas.}}, from which I cannot recover because 
> {{kafka-reassign-partitions.sh}}.
> Is there 

[jira] [Created] (KAFKA-6442) Catch 22 with cluster rebalancing

2018-01-11 Thread Andreas (JIRA)
Andreas created KAFKA-6442:
--

 Summary: Catch 22 with cluster rebalancing
 Key: KAFKA-6442
 URL: https://issues.apache.org/jira/browse/KAFKA-6442
 Project: Kafka
  Issue Type: Bug
Reporter: Andreas
 Fix For: 0.8.2.1


PS. I classified this as a bug because I think the cluster should not be stuck 
in that situation, apologies if that is wrong.

Hi,
I found myself in a situation a bit difficult to explain so I will skip the how 
I ended up in this situation, but here is the problem.

Some of the brokers of my cluster are permanently gone. Consequently, I had 
some partitions that now had offline leaders etc so, I used the 
`kafka-reassign-partitions.sh` to rebalance my topics and for the most part 
that worked ok. Where that did not work ok, was for partitions that had 
leaders, rs and irs completely in the gone brokers. Those got stuck halfway 
through to what now looks like

`Topic: topicA  Partition: 32   Leader: -1  Replicas: 1,6,2,7,3,8   Isr:`
(1,2,3 are legit, 6,7,8 permanently gone)

So the first catch 22, is that I cannot elect a new leader, because the leader 
needs to be elected from the ISR, and I cannot recreate the ISR because the 
topic has no leader.

The second catch 22 is that I cannot rerun `kafka-reassign-partitions.sh` 
because the previous one is supposedly still in progress, and I cannot increase 
the number of partitions to account for the now permanently offline partitions, 
because that produces the following error `Error while executing topic command 
requirement failed: All partitions should have the same number of replicas.`, 
from which I cannot recover because `kafka-reassign-partitions.sh`.

Is there a way to recover from such a situation? 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-6378) NullPointerException on KStream-GlobalKTable leftJoin when KeyValueMapper returns null

2018-01-11 Thread Andy Bryant (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-6378?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16322091#comment-16322091
 ] 

Andy Bryant commented on KAFKA-6378:


Sure thing Matthias. Will try and get that out tomorrow

> NullPointerException on KStream-GlobalKTable leftJoin when KeyValueMapper 
> returns null
> --
>
> Key: KAFKA-6378
> URL: https://issues.apache.org/jira/browse/KAFKA-6378
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 1.0.0
>Reporter: Andy Bryant
>
> On a Stream->GlobalKTable leftJoin if the KeyValueMapper returns null, the 
> stream fails with a NullPointerException (see stacktrace below). On Kafka 
> 0.11.0.0 the stream processes this successfully, calling the ValueJoiner with 
> the table value set to null.
> The use-case for this is joining a stream to a table containing reference 
> data where the stream foreign key may be null. There is no straight-forward 
> workaround in this case with Kafka 1.0.0 without having to resort to either 
> generating a key that will never match or branching the stream for records 
> that don't have the foreign key.
> Exception in thread "workshop-simple-example-client-StreamThread-1" 
> java.lang.NullPointerException
>   at java.base/java.util.Objects.requireNonNull(Objects.java:221)
>   at 
> org.apache.kafka.streams.state.internals.CachingKeyValueStore.get(CachingKeyValueStore.java:136)
>   at 
> org.apache.kafka.streams.state.internals.CachingKeyValueStore.get(CachingKeyValueStore.java:35)
>   at 
> org.apache.kafka.streams.state.internals.InnerMeteredKeyValueStore.get(InnerMeteredKeyValueStore.java:184)
>   at 
> org.apache.kafka.streams.state.internals.MeteredKeyValueBytesStore.get(MeteredKeyValueBytesStore.java:116)
>   at 
> org.apache.kafka.streams.kstream.internals.KTableSourceValueGetterSupplier$KTableSourceValueGetter.get(KTableSourceValueGetterSupplier.java:49)
>   at 
> org.apache.kafka.streams.kstream.internals.KStreamKTableJoinProcessor.process(KStreamKTableJoinProcessor.java:56)
>   at 
> org.apache.kafka.streams.processor.internals.ProcessorNode$1.run(ProcessorNode.java:46)
>   at 
> org.apache.kafka.streams.processor.internals.StreamsMetricsImpl.measureLatencyNs(StreamsMetricsImpl.java:208)
>   at 
> org.apache.kafka.streams.processor.internals.ProcessorNode.process(ProcessorNode.java:124)
>   at 
> org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:85)
>   at 
> org.apache.kafka.streams.processor.internals.SourceNode.process(SourceNode.java:80)
>   at 
> org.apache.kafka.streams.processor.internals.StreamTask.process(StreamTask.java:216)
>   at 
> org.apache.kafka.streams.processor.internals.AssignedTasks.process(AssignedTasks.java:403)
>   at 
> org.apache.kafka.streams.processor.internals.TaskManager.process(TaskManager.java:317)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.processAndMaybeCommit(StreamThread.java:942)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:822)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:774)
>   at 
> org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:744)



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-5504) Kafka controller is not getting elected

2018-01-11 Thread Andrey (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-5504?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16322036#comment-16322036
 ] 

Andrey commented on KAFKA-5504:
---

Hi,

Yes, see my previous comment. 
https://issues.apache.org/jira/browse/KAFKA-5504?focusedCommentId=16251161=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16251161

We put authentication for zookeeper cluster, but haven't changed already 
created Kafka znodes.

> Kafka controller is not getting elected
> ---
>
> Key: KAFKA-5504
> URL: https://issues.apache.org/jira/browse/KAFKA-5504
> Project: Kafka
>  Issue Type: Bug
>  Components: controller
>Affects Versions: 0.9.0.1
>Reporter: Ashish Kumar
>
> I am having a kafka cluster of 20 nodes and I was facing the issue of 
> under-replicated topics issue for last few days so decided to restart the 
> broker which was working as a controller but after restart getting below logs 
> in all the brokers (It seems controller is not finalized and leader election 
> is happening continuously):
> [2017-06-23 02:59:50,388] INFO Result of znode creation is: NODEEXISTS 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:50,396] INFO New leader is 12 
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> [2017-06-23 02:59:50,410] INFO Rolled new log segment for 
> 'dpe_feedback_rating_history-4' in 0 ms. (kafka.log.Log)
> [2017-06-23 02:59:51,585] INFO Creating /controller (is it secure? false) 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:51,590] INFO Result of znode creation is: NODEEXISTS 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:51,609] INFO New leader is 11 
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> [2017-06-23 02:59:52,792] INFO Creating /controller (is it secure? false) 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:52,799] INFO Result of znode creation is: NODEEXISTS 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:52,808] INFO New leader is 12 
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> [2017-06-23 02:59:54,122] INFO New leader is 3 
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> [2017-06-23 02:59:55,504] INFO Creating /controller (is it secure? false) 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:55,512] INFO Result of znode creation is: NODEEXISTS 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:55,520] INFO New leader is 11 
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> [2017-06-23 02:59:56,695] INFO Creating /controller (is it secure? false) 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:56,701] INFO Result of znode creation is: NODEEXISTS 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:56,709] INFO New leader is 11 
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> [2017-06-23 02:59:57,949] INFO Creating /controller (is it secure? false) 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:57,955] INFO Result of znode creation is: NODEEXISTS 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:57,965] INFO New leader is 12 
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> [2017-06-23 02:59:59,378] INFO Creating /controller (is it secure? false) 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:59,384] INFO Result of znode creation is: NODEEXISTS 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:59,395] INFO New leader is 12 
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> .
> .
> .
> Tried deleting controller znode (/controller) but no luck. Please let me know 
> if any fix is possible here.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-5504) Kafka controller is not getting elected

2018-01-11 Thread Lakshmi G S (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-5504?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16322027#comment-16322027
 ] 

Lakshmi G S commented on KAFKA-5504:


Hi Andrey,

Have you identified the root cause yet? We are facing the same issue in one of 
our environments.

> Kafka controller is not getting elected
> ---
>
> Key: KAFKA-5504
> URL: https://issues.apache.org/jira/browse/KAFKA-5504
> Project: Kafka
>  Issue Type: Bug
>  Components: controller
>Affects Versions: 0.9.0.1
>Reporter: Ashish Kumar
>
> I am having a kafka cluster of 20 nodes and I was facing the issue of 
> under-replicated topics issue for last few days so decided to restart the 
> broker which was working as a controller but after restart getting below logs 
> in all the brokers (It seems controller is not finalized and leader election 
> is happening continuously):
> [2017-06-23 02:59:50,388] INFO Result of znode creation is: NODEEXISTS 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:50,396] INFO New leader is 12 
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> [2017-06-23 02:59:50,410] INFO Rolled new log segment for 
> 'dpe_feedback_rating_history-4' in 0 ms. (kafka.log.Log)
> [2017-06-23 02:59:51,585] INFO Creating /controller (is it secure? false) 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:51,590] INFO Result of znode creation is: NODEEXISTS 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:51,609] INFO New leader is 11 
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> [2017-06-23 02:59:52,792] INFO Creating /controller (is it secure? false) 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:52,799] INFO Result of znode creation is: NODEEXISTS 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:52,808] INFO New leader is 12 
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> [2017-06-23 02:59:54,122] INFO New leader is 3 
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> [2017-06-23 02:59:55,504] INFO Creating /controller (is it secure? false) 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:55,512] INFO Result of znode creation is: NODEEXISTS 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:55,520] INFO New leader is 11 
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> [2017-06-23 02:59:56,695] INFO Creating /controller (is it secure? false) 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:56,701] INFO Result of znode creation is: NODEEXISTS 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:56,709] INFO New leader is 11 
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> [2017-06-23 02:59:57,949] INFO Creating /controller (is it secure? false) 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:57,955] INFO Result of znode creation is: NODEEXISTS 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:57,965] INFO New leader is 12 
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> [2017-06-23 02:59:59,378] INFO Creating /controller (is it secure? false) 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:59,384] INFO Result of znode creation is: NODEEXISTS 
> (kafka.utils.ZKCheckedEphemeral)
> [2017-06-23 02:59:59,395] INFO New leader is 12 
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> .
> .
> .
> Tried deleting controller znode (/controller) but no luck. Please let me know 
> if any fix is possible here.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-6323) punctuate with WALL_CLOCK_TIME triggered immediately

2018-01-11 Thread Frederic Arno (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-6323?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16321878#comment-16321878
 ] 

Frederic Arno commented on KAFKA-6323:
--

I agree and the PR doesn't need to be updated for the above, as it already 
works like that (see the tests).

> punctuate with WALL_CLOCK_TIME triggered immediately
> 
>
> Key: KAFKA-6323
> URL: https://issues.apache.org/jira/browse/KAFKA-6323
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 1.0.0
>Reporter: Frederic Arno
>Assignee: Frederic Arno
> Fix For: 1.1.0, 1.0.1
>
>
> When working on a custom Processor from which I am scheduling a punctuation 
> using WALL_CLOCK_TIME. I've noticed that whatever the punctuation interval I 
> set, a call to my Punctuator is always triggered immediately.
> Having a quick look at kafka-streams' code, I could find that all 
> PunctuationSchedule's timestamps are matched against the current time in 
> order to decide whether or not to trigger the punctuator 
> (org.apache.kafka.streams.processor.internals.PunctuationQueue#mayPunctuate). 
> However, I've only seen code that initializes PunctuationSchedule's timestamp 
> to 0, which I guess is what is causing an immediate punctuation.
> At least when using WALL_CLOCK_TIME, shouldn't the PunctuationSchedule's 
> timestamp be initialized to current time + interval?



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)