[kafka] branch trunk updated: MINOR: fix bypasses in ChangeLogging stores (#6266)

2019-02-14 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new af91eee  MINOR: fix bypasses in ChangeLogging stores (#6266)
af91eee is described below

commit af91eeeb793af2f5b873c48ef8ff636e8b485308
Author: John Roesler 
AuthorDate: Thu Feb 14 12:39:10 2019 -0600

MINOR: fix bypasses in ChangeLogging stores (#6266)

The change-logging stores should not bypass methods in underlying stores.

If some of you have a minute, can you take a quick look at this? I happened 
to notice during some other refactoring that the change-logging store layer 
sometimes bypasses the underlying store and instead calls across to a different 
layer.

It seems unexpected that it should do so, and it might actually cause 
problems. There was one spot where it's impossible to avoid it (in the windowed 
store), but I added a note justifying why we bypass the underlying store.

Thanks,
-John

* MINOR: fix bypasses in ChangeLogging stores

* fix test

Reviewers: Guozhang Wang , Matthias J. Sax 
, Bill Bejeck 
---
 .../streams/state/internals/ChangeLoggingKeyValueBytesStore.java   | 7 ---
 .../streams/state/internals/ChangeLoggingSessionBytesStore.java| 4 ++--
 .../streams/state/internals/ChangeLoggingWindowBytesStore.java | 4 
 .../state/internals/ChangeLoggingSessionBytesStoreTest.java| 4 ++--
 4 files changed, 12 insertions(+), 7 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java
index d5f5ad2..7567e78 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java
@@ -40,7 +40,7 @@ public class ChangeLoggingKeyValueBytesStore extends 
WrappedStateStore(name(), context, new 
StateSerdes<>(topic, Serdes.Bytes(), Serdes.ByteArray()));
+changeLogger = new StoreChangeLogger<>(name(), context, new 
StateSerdes<>(topic, Serdes.Bytes(), Serdes.ByteArray()));
 
 // if the inner store is an LRU cache, add the eviction listener to 
log removed record
 if (wrapped() instanceof MemoryLRUCache) {
@@ -66,9 +66,10 @@ public class ChangeLoggingKeyValueBytesStore extends 
WrappedStateStore, byte[]> fetch(final Bytes key) {
-return findSessions(key, 0, Long.MAX_VALUE);
+return wrapped().fetch(key);
 }
 
 @Override
 public KeyValueIterator, byte[]> fetch(final Bytes from, 
final Bytes to) {
-return findSessions(from, to, 0, Long.MAX_VALUE);
+return wrapped().fetch(from, to);
 }
 }
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java
index a614f92..3cddb33 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java
@@ -74,6 +74,10 @@ class ChangeLoggingWindowBytesStore extends 
WrappedStateStore, 
byte[]>emptyIterator());
+
EasyMock.expect(inner.fetch(bytesKey)).andReturn(KeyValueIterators.,
 byte[]>emptyIterator());
 
 init();
 
@@ -123,7 +123,7 @@ public class ChangeLoggingSessionBytesStoreTest {
 
 @Test
 public void shouldDelegateToUnderlyingStoreWhenFetchingRange() {
-EasyMock.expect(inner.findSessions(bytesKey, bytesKey, 0, 
Long.MAX_VALUE)).andReturn(KeyValueIterators., 
byte[]>emptyIterator());
+EasyMock.expect(inner.fetch(bytesKey, 
bytesKey)).andReturn(KeyValueIterators., 
byte[]>emptyIterator());
 
 init();
 



[kafka] branch trunk updated: MINOR: add test for StreamsSmokeTestDriver (#6231)

2019-02-15 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 3656ad9  MINOR: add test for StreamsSmokeTestDriver (#6231)
3656ad9 is described below

commit 3656ad93bc7fb74138e36d39575c3b86c319577b
Author: John Roesler 
AuthorDate: Fri Feb 15 10:23:57 2019 -0600

MINOR: add test for StreamsSmokeTestDriver (#6231)

* MINOR: add test for StreamsSmokeTestDriver
Hi @bbejeck @mjsax @ableegoldman @guozhangwang ,

Please take a look at this when you get the chance.

The primary concern is adding the test. It will help us verify changes to 
the smoke test (such as adding suppression).

I've also added some extra output to the smoke test stdout, which will 
hopefully aid us in diagnosing the flaky tests.

Finally, I bundled in some cleanup. It was my intention to do that in a 
separate PR, but it wound up getting smashed together during refactoring.

Please let me know if you'd prefer for me to pull any of these out into a 
separate request.

Thanks,
-John

Also, add more output for debuggability

* cleanup

* cleanup

* refactor

* refactor

* remove redundant printlns

* Update EmbeddedKafkaCluster.java

* move to integration package

* replace early-exit on pass

* use classrule for embedded kafka

* pull in smoke test improvements from side branch

* try-with-resources

* format events instead of printing long lines

* minor formatting fix

Reviewers:  Matthias J. Sax , Bill Bejeck 

---
 .../SmokeTestDriverIntegrationTest.java| 134 +
 .../kafka/streams/tests/SmokeTestClient.java   | 189 +++
 .../kafka/streams/tests/SmokeTestDriver.java   | 564 -
 .../apache/kafka/streams/tests/SmokeTestUtil.java  |   7 +-
 .../kafka/streams/tests/StreamsSmokeTest.java  |  10 +-
 5 files changed, 438 insertions(+), 466 deletions(-)

diff --git 
a/streams/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java
new file mode 100644
index 000..82f86c2
--- /dev/null
+++ 
b/streams/src/test/java/org/apache/kafka/streams/integration/SmokeTestDriverIntegrationTest.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.integration;
+
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.tests.SmokeTestClient;
+import org.apache.kafka.streams.tests.SmokeTestDriver;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import static org.apache.kafka.streams.tests.SmokeTestDriver.generate;
+import static org.apache.kafka.streams.tests.SmokeTestDriver.verify;
+
+public class SmokeTestDriverIntegrationTest {
+@ClassRule
+public static final EmbeddedKafkaCluster CLUSTER = new 
EmbeddedKafkaCluster(3);
+
+
+private static class Driver extends Thread {
+private String bootstrapServers;
+private int numKeys;
+private int maxRecordsPerKey;
+private Exception exception = null;
+private SmokeTestDriver.VerificationResult result;
+
+private Driver(final String bootstrapServers, final int numKeys, final 
int maxRecordsPerKey) {
+this.bootstrapServers = bootstrapServers;
+this.numKeys = numKeys;
+this.maxRecordsPerKey = maxRecordsPerKey;
+}
+
+@Override
+public void run() {
+try {
+final Map> allData = 
generate(bootstrapServers, numKeys, maxRecordsPerKey, true);
+result = verify(bootstrapServers, allData, maxRecordsPerKey);
+
+} catch (final Exception ex) {
+  

[kafka-site] branch asf-site updated: Add committer picture and info for Bill Bejeck

2019-02-17 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/kafka-site.git


The following commit(s) were added to refs/heads/asf-site by this push:
 new 813e576  Add committer picture and info for Bill Bejeck
813e576 is described below

commit 813e576bcb0536e42cda56bbf5b90985f005c1dc
Author: Bill Bejeck 
AuthorDate: Sun Feb 17 20:07:12 2019 -0500

Add committer picture and info for Bill Bejeck
---
 committers.html |   5 +
 images/bbejeck_pic.jpeg | Bin 0 -> 45300 bytes
 2 files changed, 5 insertions(+)

diff --git a/committers.html b/committers.html
index 4729edd..b2f96a2 100644
--- a/committers.html
+++ b/committers.html
@@ -249,8 +249,13 @@
   https://twitter.com/vahidh";>@vahidh
 
 
+  
 
 
+  Bill Bejeck
+  Committer
+  https://www.linkedin.com/in/bbejeck/";>/in/bbejeck
+  https://twitter.com/bbejeck";>@bbejeck
 
   
 
diff --git a/images/bbejeck_pic.jpeg b/images/bbejeck_pic.jpeg
new file mode 100644
index 000..de96e1d
Binary files /dev/null and b/images/bbejeck_pic.jpeg differ



[kafka] branch trunk updated: MINOR: improve JavaDocs about auto-repartitioning in Streams DSL (#6269)

2019-02-18 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 9fe89f3  MINOR: improve JavaDocs about auto-repartitioning in Streams 
DSL (#6269)
9fe89f3 is described below

commit 9fe89f357ced1f75d5c7053979e8ec6430041885
Author: Matthias J. Sax 
AuthorDate: Mon Feb 18 11:22:41 2019 -0800

MINOR: improve JavaDocs about auto-repartitioning in Streams DSL (#6269)
---
 .../org/apache/kafka/streams/kstream/KStream.java  | 59 +++---
 1 file changed, 19 insertions(+), 40 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java 
b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
index 44778f0..5138917 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
@@ -95,7 +95,6 @@ public interface KStream {
  * }
  * });
  * }
- * 
  * Setting a new key might result in an internal data redistribution if a 
key based operator (like an aggregation or
  * join) is applied to the result {@code KStream}.
  *
@@ -128,7 +127,6 @@ public interface KStream {
  * }
  * });
  * }
- * 
  * The provided {@link KeyValueMapper} must return a {@link KeyValue} type 
and must not return {@code null}.
  * 
  * Mapping records might result in an internal data redistribution if a 
key based operator (like an aggregation or
@@ -166,7 +164,6 @@ public interface KStream {
  * }
  * });
  * }
- * 
  * Setting a new value preserves data co-location with respect to the key.
  * Thus, no internal data redistribution is required if a key 
based operator (like an aggregation or join)
  * is applied to the result {@code KStream}. (cf. {@link 
#map(KeyValueMapper)})
@@ -201,7 +198,6 @@ public interface KStream {
  * }
  * });
  * }
- * 
  * Note that the key is read-only and should not be modified, as this can 
lead to corrupt partitioning.
  * So, setting a new value preserves data co-location with respect to the 
key.
  * Thus, no internal data redistribution is required if a key 
based operator (like an aggregation or join)
@@ -246,7 +242,6 @@ public interface KStream {
  * }
  * });
  * }
- * 
  * The provided {@link KeyValueMapper} must return an {@link Iterable} 
(e.g., any {@link java.util.Collection} type)
  * and the return value must not be {@code null}.
  * 
@@ -289,7 +284,6 @@ public interface KStream {
  * }
  * });
  * }
- * 
  * The provided {@link ValueMapper} must return an {@link Iterable} (e.g., 
any {@link java.util.Collection} type)
  * and the return value must not be {@code null}.
  * 
@@ -336,7 +330,6 @@ public interface KStream {
  * }
  * });
  * }
- * 
  * The provided {@link ValueMapperWithKey} must return an {@link Iterable} 
(e.g., any {@link java.util.Collection} type)
  * and the return value must not be {@code null}.
  * 
@@ -503,7 +496,6 @@ public interface KStream {
  * This is a stateful record-by-record operation (cf. {@link 
#map(KeyValueMapper)}).
  * Furthermore, via {@link 
org.apache.kafka.streams.processor.Punctuator#punctuate(long)}, the processing 
progress
  * can be observed and additional periodic actions can be performed.
- *
  * 
  * In order to assign a state, the state must be created and registered 
beforehand:
  * {@code
@@ -517,7 +509,6 @@ public interface KStream {
  *
  * KStream outputStream = inputStream.transform(new TransformerSupplier() 
{ ... }, "myTransformState");
  * }
- * 
  * Within the {@link Transformer}, the state is obtained via the {@link  
ProcessorContext}.
  * To trigger periodic actions via {@link 
org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()},
  * a schedule must be registered.
@@ -549,12 +540,12 @@ public interface KStream {
  * }
  * }
  * }
+ * Even if any upstream operation was key-changing, no auto-repartition is 
triggered.
+ * If repartitioning is required, a call to {@link #through(String)} 
should be performed before {@code transform()}.
  * 
  * Transforming records might result in an internal data redistribution if 
a key based operator (like an aggregation
  * or join) is applied to the result {@code KStream}.
  * (cf. {@link #transformValues(ValueTransformerSupplier, String...)})
- * 
- *
  * 
  * Note that it is possible to emit multiple records for each input record 
by using
  * {@link ProcessorContext#forward(Object, Object) context#forward()} in 
{@link Transformer#transform(K, V)}.
@@ -563,7 +554,6 @@ public interfa

[kafka] branch 2.2 updated: MINOR: improve JavaDocs about auto-repartitioning in Streams DSL (#6269)

2019-02-18 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new 3c3b0ed  MINOR: improve JavaDocs about auto-repartitioning in Streams 
DSL (#6269)
3c3b0ed is described below

commit 3c3b0ed84a5ff4ed026d9466c6c84083a63fb5c0
Author: Matthias J. Sax 
AuthorDate: Mon Feb 18 11:22:41 2019 -0800

MINOR: improve JavaDocs about auto-repartitioning in Streams DSL (#6269)
---
 .../org/apache/kafka/streams/kstream/KStream.java  | 59 +++---
 1 file changed, 19 insertions(+), 40 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java 
b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
index 44778f0..5138917 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
@@ -95,7 +95,6 @@ public interface KStream {
  * }
  * });
  * }
- * 
  * Setting a new key might result in an internal data redistribution if a 
key based operator (like an aggregation or
  * join) is applied to the result {@code KStream}.
  *
@@ -128,7 +127,6 @@ public interface KStream {
  * }
  * });
  * }
- * 
  * The provided {@link KeyValueMapper} must return a {@link KeyValue} type 
and must not return {@code null}.
  * 
  * Mapping records might result in an internal data redistribution if a 
key based operator (like an aggregation or
@@ -166,7 +164,6 @@ public interface KStream {
  * }
  * });
  * }
- * 
  * Setting a new value preserves data co-location with respect to the key.
  * Thus, no internal data redistribution is required if a key 
based operator (like an aggregation or join)
  * is applied to the result {@code KStream}. (cf. {@link 
#map(KeyValueMapper)})
@@ -201,7 +198,6 @@ public interface KStream {
  * }
  * });
  * }
- * 
  * Note that the key is read-only and should not be modified, as this can 
lead to corrupt partitioning.
  * So, setting a new value preserves data co-location with respect to the 
key.
  * Thus, no internal data redistribution is required if a key 
based operator (like an aggregation or join)
@@ -246,7 +242,6 @@ public interface KStream {
  * }
  * });
  * }
- * 
  * The provided {@link KeyValueMapper} must return an {@link Iterable} 
(e.g., any {@link java.util.Collection} type)
  * and the return value must not be {@code null}.
  * 
@@ -289,7 +284,6 @@ public interface KStream {
  * }
  * });
  * }
- * 
  * The provided {@link ValueMapper} must return an {@link Iterable} (e.g., 
any {@link java.util.Collection} type)
  * and the return value must not be {@code null}.
  * 
@@ -336,7 +330,6 @@ public interface KStream {
  * }
  * });
  * }
- * 
  * The provided {@link ValueMapperWithKey} must return an {@link Iterable} 
(e.g., any {@link java.util.Collection} type)
  * and the return value must not be {@code null}.
  * 
@@ -503,7 +496,6 @@ public interface KStream {
  * This is a stateful record-by-record operation (cf. {@link 
#map(KeyValueMapper)}).
  * Furthermore, via {@link 
org.apache.kafka.streams.processor.Punctuator#punctuate(long)}, the processing 
progress
  * can be observed and additional periodic actions can be performed.
- *
  * 
  * In order to assign a state, the state must be created and registered 
beforehand:
  * {@code
@@ -517,7 +509,6 @@ public interface KStream {
  *
  * KStream outputStream = inputStream.transform(new TransformerSupplier() 
{ ... }, "myTransformState");
  * }
- * 
  * Within the {@link Transformer}, the state is obtained via the {@link  
ProcessorContext}.
  * To trigger periodic actions via {@link 
org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()},
  * a schedule must be registered.
@@ -549,12 +540,12 @@ public interface KStream {
  * }
  * }
  * }
+ * Even if any upstream operation was key-changing, no auto-repartition is 
triggered.
+ * If repartitioning is required, a call to {@link #through(String)} 
should be performed before {@code transform()}.
  * 
  * Transforming records might result in an internal data redistribution if 
a key based operator (like an aggregation
  * or join) is applied to the result {@code KStream}.
  * (cf. {@link #transformValues(ValueTransformerSupplier, String...)})
- * 
- *
  * 
  * Note that it is possible to emit multiple records for each input record 
by using
  * {@link ProcessorContext#forward(Object, Object) context#forward()} in 
{@link Transformer#transform(K, V)}.
@@ -563,7 +554,6 @@ public interface KStream {

[kafka-site] branch asf-site updated: Correct the name of the generated directory when using the maven archetype as described in the "Write a Streams Application" tutorial (#183)

2019-02-19 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/kafka-site.git


The following commit(s) were added to refs/heads/asf-site by this push:
 new 5fb61fb  Correct the name of the generated directory when using the 
maven archetype as described in the "Write a Streams Application" tutorial 
(#183)
5fb61fb is described below

commit 5fb61fb0f7f609b85978da65793a6ee7935ef6d0
Author: Kristian Aurlien 
AuthorDate: Tue Feb 19 20:54:16 2019 +0100

Correct the name of the generated directory when using the maven archetype 
as described in the "Write a Streams Application" tutorial (#183)

In the streams tutorial, the code examples mixes `streams-quickstart` and 
`streams.examples`. I believe that they should all be `streams.examples`.

Affected version: 2.1._, 2.0._, 1.1._, 1.0._

Tested on mac os x v10.14.2, Java v1.8, Maven v3.5.4

Reviewers: Bill Bejeck ,  John Roesler 

---
 10/streams/tutorial.html | 4 ++--
 11/streams/tutorial.html | 4 ++--
 20/streams/tutorial.html | 4 ++--
 21/streams/tutorial.html | 4 ++--
 4 files changed, 8 insertions(+), 8 deletions(-)

diff --git a/10/streams/tutorial.html b/10/streams/tutorial.html
index 0bad0e5..424947f 100644
--- a/10/streams/tutorial.html
+++ b/10/streams/tutorial.html
@@ -60,7 +60,7 @@
 
 
 > tree streams.examples
-streams-quickstart
+streams.examples
 |-- pom.xml
 |-- src
 |-- main
@@ -80,7 +80,7 @@
 
 
 
-> cd streams-quickstart
+> cd streams.examples
 > rm src/main/java/myapps/*.java
 
 
diff --git a/11/streams/tutorial.html b/11/streams/tutorial.html
index 21ff030..05e8217 100644
--- a/11/streams/tutorial.html
+++ b/11/streams/tutorial.html
@@ -60,7 +60,7 @@
 
 
 > tree streams.examples
-streams-quickstart
+streams.examples
 |-- pom.xml
 |-- src
 |-- main
@@ -80,7 +80,7 @@
 
 
 
-> cd streams-quickstart
+> cd streams.examples
 > rm src/main/java/myapps/*.java
 
 
diff --git a/20/streams/tutorial.html b/20/streams/tutorial.html
index 0006e3e..d13f4e7 100644
--- a/20/streams/tutorial.html
+++ b/20/streams/tutorial.html
@@ -60,7 +60,7 @@
 
 
 > tree streams.examples
-streams-quickstart
+streams.examples
 |-- pom.xml
 |-- src
 |-- main
@@ -80,7 +80,7 @@
 
 
 
-> cd streams-quickstart
+> cd streams.examples
 > rm src/main/java/myapps/*.java
 
 
diff --git a/21/streams/tutorial.html b/21/streams/tutorial.html
index 0006e3e..d13f4e7 100644
--- a/21/streams/tutorial.html
+++ b/21/streams/tutorial.html
@@ -60,7 +60,7 @@
 
 
 > tree streams.examples
-streams-quickstart
+streams.examples
 |-- pom.xml
 |-- src
 |-- main
@@ -80,7 +80,7 @@
 
 
 
-> cd streams-quickstart
+> cd streams.examples
 > rm src/main/java/myapps/*.java
 
 



[kafka] branch 2.2 updated: KAFKA-7895: fix stream-time reckoning for Suppress (2.2) (#6286)

2019-02-20 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new 7517d4e  KAFKA-7895: fix stream-time reckoning for Suppress (2.2) 
(#6286)
7517d4e is described below

commit 7517d4e114cf772abfb918fbfa1ed35e3c5a404d
Author: John Roesler 
AuthorDate: Wed Feb 20 09:15:57 2019 -0600

KAFKA-7895: fix stream-time reckoning for Suppress (2.2) (#6286)

Even within a Task, different Processors have different perceptions
of time, due to record caching on stores and in suppression itself,
and in general, due to any processor logic that may hold onto
records arbitrarily and emit them later. Thanks to this, we can't rely
on the whole task existing in the same "instant" of stream-time. The
solution is for each processor node that cares about stream-time to
track it independently.

Reviewers: Matthias J. Sax , Bill Bejeck 

---
 .../internals/KStreamSessionWindowAggregate.java   |   5 +-
 .../kstream/internals/KStreamWindowAggregate.java  |   5 +-
 .../suppress/KTableSuppressProcessor.java  |   8 +-
 .../internals/GlobalProcessorContextImpl.java  |   8 --
 .../internals/InternalProcessorContext.java|   2 -
 .../processor/internals/ProcessorContextImpl.java  |  10 --
 .../streams/processor/internals/RecordQueue.java   |   2 +-
 .../processor/internals/StandbyContextImpl.java|  12 --
 .../streams/processor/internals/StandbyTask.java   |   8 +-
 .../streams/processor/internals/StreamTask.java|   1 -
 .../processor/internals/TimestampSupplier.java |  21 ---
 .../streams/state/internals/AbstractSegments.java  |  10 +-
 .../internals/RocksDBSegmentedBytesStore.java  |  24 +++-
 .../kafka/streams/state/internals/Segments.java|   4 +-
 ...KStreamSessionWindowAggregateProcessorTest.java |   6 +-
 .../KTableSuppressProcessorMetricsTest.java|   2 -
 .../suppress/KTableSuppressProcessorTest.java  |  19 ---
 .../internals/AbstractProcessorContextTest.java|   5 -
 .../internals/GlobalProcessorContextImplTest.java  |   5 -
 .../state/internals/KeyValueSegmentsTest.java  |  72 +-
 .../internals/RocksDBSegmentedBytesStoreTest.java  |  20 ++-
 .../state/internals/RocksDBWindowStoreTest.java|   5 +-
 .../kafka/streams/tests/SmokeTestClient.java   | 147 +
 .../kafka/streams/tests/SmokeTestDriver.java   |  58 +++-
 .../kafka/test/InternalMockProcessorContext.java   |  14 +-
 .../kafka/test/MockInternalProcessorContext.java   |  10 --
 .../apache/kafka/test/NoOpProcessorContext.java|   5 -
 27 files changed, 224 insertions(+), 264 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java
index 09e1cb4..60f1b6a 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.kstream.internals;
 
+import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.kstream.Aggregator;
@@ -81,6 +82,7 @@ public class KStreamSessionWindowAggregate 
implements KStreamAggProce
 private StreamsMetricsImpl metrics;
 private InternalProcessorContext internalProcessorContext;
 private Sensor lateRecordDropSensor;
+private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
 
 @SuppressWarnings("unchecked")
 @Override
@@ -107,7 +109,8 @@ public class KStreamSessionWindowAggregate 
implements KStreamAggProce
 return;
 }
 
-final long closeTime = internalProcessorContext.streamTime() - 
windows.gracePeriodMs();
+observedStreamTime = Math.max(observedStreamTime, 
context().timestamp());
+final long closeTime = observedStreamTime - 
windows.gracePeriodMs();
 
 final long timestamp = context().timestamp();
 final List, Agg>> merged = new ArrayList<>();
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
index 0edbe4e..e5f290e 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka

[kafka] branch trunk updated: MINOR: Add all topics created check streams broker bounce test (trunk) (#6243)

2019-02-20 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new be76560  MINOR: Add all topics created check streams broker bounce 
test (trunk) (#6243)
be76560 is described below

commit be765600111246430f4d61689e068cc55a1ade7b
Author: Bill Bejeck 
AuthorDate: Wed Feb 20 12:45:22 2019 -0500

MINOR: Add all topics created check streams broker bounce test (trunk) 
(#6243)

The StreamsBrokerBounceTest.test_broker_type_bounce experienced what looked 
like a transient failure. After looking over this test and failure, it seems 
like it is vulnerable to timing error that streams will start before the kafka 
service creates all topics.

Reviewers:  Matthias J. Sax , John Roesler 

---
 .../tests/streams/streams_broker_bounce_test.py| 22 ++
 1 file changed, 22 insertions(+)

diff --git a/tests/kafkatest/tests/streams/streams_broker_bounce_test.py 
b/tests/kafkatest/tests/streams/streams_broker_bounce_test.py
index 8d623eb..7859d69 100644
--- a/tests/kafkatest/tests/streams/streams_broker_bounce_test.py
+++ b/tests/kafkatest/tests/streams/streams_broker_bounce_test.py
@@ -133,6 +133,22 @@ class StreamsBrokerBounceTest(Test):
 for num in range(0, num_failures - 1):
 signal_node(self, self.kafka.nodes[num], sig)
 
+def confirm_topics_on_all_brokers(self, expected_topic_set):
+for node in self.kafka.nodes:
+match_count = 0
+# need to iterate over topic_list_generator as kafka.list_topics()
+# returns a python generator so values are fetched lazily
+# so we can't just compare directly we must iterate over what's 
returned
+topic_list_generator = self.kafka.list_topics("placeholder", node)
+for topic in topic_list_generator:
+if topic in expected_topic_set:
+match_count += 1
+
+if len(expected_topic_set) != match_count:
+return False
+
+return True
+
 
 def setup_system(self, start_processor=True):
 # Setup phase
@@ -141,6 +157,12 @@ class StreamsBrokerBounceTest(Test):
 
 self.kafka = KafkaService(self.test_context, 
num_nodes=self.replication, zk=self.zk, topics=self.topics)
 self.kafka.start()
+
+# allow some time for topics to be created
+wait_until(lambda: 
self.confirm_topics_on_all_brokers(set(self.topics.keys())),
+   timeout_sec=60,
+   err_msg="Broker did not create all topics in 60 seconds ")
+
 # Start test harness
 self.driver = StreamsSmokeTestDriverService(self.test_context, 
self.kafka)
 self.processor1 = StreamsSmokeTestJobRunnerService(self.test_context, 
self.kafka)



[kafka] branch 2.2 updated: MINOR: Add all topics created check streams broker bounce test (2.2) (#6244)

2019-02-20 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new 034968b  MINOR: Add all topics created check streams broker bounce 
test (2.2) (#6244)
034968b is described below

commit 034968b1ac5edcc3e4f80bb98a2a8d20e5a82022
Author: Bill Bejeck 
AuthorDate: Wed Feb 20 12:47:05 2019 -0500

MINOR: Add all topics created check streams broker bounce test (2.2) (#6244)

The StreamsBrokerBounceTest.test_broker_type_bounce experienced what looked 
like a transient failure. After looking over this test and failure, it seems 
like it is vulnerable to timing error that streams will start before the kafka 
service creates all topics.

Reviews:  Matthias J. Sax , John Roesler 

---
 .../tests/streams/streams_broker_bounce_test.py| 22 ++
 1 file changed, 22 insertions(+)

diff --git a/tests/kafkatest/tests/streams/streams_broker_bounce_test.py 
b/tests/kafkatest/tests/streams/streams_broker_bounce_test.py
index 8d623eb..7859d69 100644
--- a/tests/kafkatest/tests/streams/streams_broker_bounce_test.py
+++ b/tests/kafkatest/tests/streams/streams_broker_bounce_test.py
@@ -133,6 +133,22 @@ class StreamsBrokerBounceTest(Test):
 for num in range(0, num_failures - 1):
 signal_node(self, self.kafka.nodes[num], sig)
 
+def confirm_topics_on_all_brokers(self, expected_topic_set):
+for node in self.kafka.nodes:
+match_count = 0
+# need to iterate over topic_list_generator as kafka.list_topics()
+# returns a python generator so values are fetched lazily
+# so we can't just compare directly we must iterate over what's 
returned
+topic_list_generator = self.kafka.list_topics("placeholder", node)
+for topic in topic_list_generator:
+if topic in expected_topic_set:
+match_count += 1
+
+if len(expected_topic_set) != match_count:
+return False
+
+return True
+
 
 def setup_system(self, start_processor=True):
 # Setup phase
@@ -141,6 +157,12 @@ class StreamsBrokerBounceTest(Test):
 
 self.kafka = KafkaService(self.test_context, 
num_nodes=self.replication, zk=self.zk, topics=self.topics)
 self.kafka.start()
+
+# allow some time for topics to be created
+wait_until(lambda: 
self.confirm_topics_on_all_brokers(set(self.topics.keys())),
+   timeout_sec=60,
+   err_msg="Broker did not create all topics in 60 seconds ")
+
 # Start test harness
 self.driver = StreamsSmokeTestDriverService(self.test_context, 
self.kafka)
 self.processor1 = StreamsSmokeTestJobRunnerService(self.test_context, 
self.kafka)



[kafka] branch 2.0 updated: MINOR: Add check all topics created check streams broker bounce test (2.0) (#6241)

2019-02-20 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.0
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.0 by this push:
 new d072db1  MINOR: Add check all topics created check streams broker 
bounce test (2.0) (#6241)
d072db1 is described below

commit d072db1d67c59436bf65a192245f423e87399eda
Author: Bill Bejeck 
AuthorDate: Wed Feb 20 16:40:33 2019 -0500

MINOR: Add check all topics created check streams broker bounce test (2.0) 
(#6241)

The StreamsBrokerBounceTest.test_broker_type_bounce experienced what looked 
like a transient failure. After looking over this test and failure, it seems 
like it is vulnerable to timing error that streams will start before the kafka 
service creates all topics.

Reviewers: Guozhang Wang , Matthias J. Sax 
, John Roesler 
---
 .../tests/streams/streams_broker_bounce_test.py| 22 ++
 1 file changed, 22 insertions(+)

diff --git a/tests/kafkatest/tests/streams/streams_broker_bounce_test.py 
b/tests/kafkatest/tests/streams/streams_broker_bounce_test.py
index 8d623eb..7859d69 100644
--- a/tests/kafkatest/tests/streams/streams_broker_bounce_test.py
+++ b/tests/kafkatest/tests/streams/streams_broker_bounce_test.py
@@ -133,6 +133,22 @@ class StreamsBrokerBounceTest(Test):
 for num in range(0, num_failures - 1):
 signal_node(self, self.kafka.nodes[num], sig)
 
+def confirm_topics_on_all_brokers(self, expected_topic_set):
+for node in self.kafka.nodes:
+match_count = 0
+# need to iterate over topic_list_generator as kafka.list_topics()
+# returns a python generator so values are fetched lazily
+# so we can't just compare directly we must iterate over what's 
returned
+topic_list_generator = self.kafka.list_topics("placeholder", node)
+for topic in topic_list_generator:
+if topic in expected_topic_set:
+match_count += 1
+
+if len(expected_topic_set) != match_count:
+return False
+
+return True
+
 
 def setup_system(self, start_processor=True):
 # Setup phase
@@ -141,6 +157,12 @@ class StreamsBrokerBounceTest(Test):
 
 self.kafka = KafkaService(self.test_context, 
num_nodes=self.replication, zk=self.zk, topics=self.topics)
 self.kafka.start()
+
+# allow some time for topics to be created
+wait_until(lambda: 
self.confirm_topics_on_all_brokers(set(self.topics.keys())),
+   timeout_sec=60,
+   err_msg="Broker did not create all topics in 60 seconds ")
+
 # Start test harness
 self.driver = StreamsSmokeTestDriverService(self.test_context, 
self.kafka)
 self.processor1 = StreamsSmokeTestJobRunnerService(self.test_context, 
self.kafka)



[kafka] branch trunk updated: HOT_FIX: Change flag so plain RocksDB instance returned (#6297)

2019-02-20 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 1898443  HOT_FIX: Change flag so plain RocksDB instance returned 
(#6297)
1898443 is described below

commit 18984434df5295b0940e468ffb467c47b0fd72dd
Author: Bill Bejeck 
AuthorDate: Wed Feb 20 19:09:23 2019 -0500

HOT_FIX: Change flag so plain RocksDB instance returned (#6297)

This PR fixes the issue found in the soak testing cluster regarding using 
RocksDBTimestampedStore when a regular RocksDB store should have been used.

Reviewers: Guozhang Wang , Matthias J. Sax 
, John Roesler 
---
 streams/src/main/java/org/apache/kafka/streams/state/Stores.java | 2 +-
 streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java | 4 +++-
 2 files changed, 4 insertions(+), 2 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java 
b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java
index b672f3d..91c895f 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java
@@ -84,7 +84,7 @@ public class Stores {
  */
 public static KeyValueBytesStoreSupplier persistentKeyValueStore(final 
String name) {
 Objects.requireNonNull(name, "name cannot be null");
-return new RocksDbKeyValueBytesStoreSupplier(name, true);
+return new RocksDbKeyValueBytesStoreSupplier(name, false);
 }
 
 /**
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java 
b/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java
index 1d4a849..9cc1280 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java
@@ -21,11 +21,13 @@ import 
org.apache.kafka.streams.state.internals.InMemoryKeyValueStore;
 import org.apache.kafka.streams.state.internals.MemoryNavigableLRUCache;
 import org.apache.kafka.streams.state.internals.RocksDBSessionStore;
 import org.apache.kafka.streams.state.internals.RocksDBStore;
+import org.apache.kafka.streams.state.internals.RocksDBTimestampedStore;
 import org.apache.kafka.streams.state.internals.RocksDBWindowStore;
 import org.junit.Test;
 
 import static java.time.Duration.ZERO;
 import static java.time.Duration.ofMillis;
+import static org.hamcrest.CoreMatchers.allOf;
 import static org.hamcrest.CoreMatchers.nullValue;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.core.IsInstanceOf.instanceOf;
@@ -113,7 +115,7 @@ public class StoresTest {
 
 @Test
 public void shouldCreateRocksDbStore() {
-assertThat(Stores.persistentKeyValueStore("store").get(), 
instanceOf(RocksDBStore.class));
+assertThat(Stores.persistentKeyValueStore("store").get(), 
allOf(not(instanceOf(RocksDBTimestampedStore.class)), 
instanceOf(RocksDBStore.class)));
 }
 
 @Test



[kafka] branch 2.2 updated: HOT_FIX: Change flag so plain RocksDB instance returned (#6297)

2019-02-20 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new 4829c0b  HOT_FIX: Change flag so plain RocksDB instance returned 
(#6297)
4829c0b is described below

commit 4829c0b9741813365adfaf5cdecafbfcd1fcc437
Author: Bill Bejeck 
AuthorDate: Wed Feb 20 19:09:23 2019 -0500

HOT_FIX: Change flag so plain RocksDB instance returned (#6297)

This PR fixes the issue found in the soak testing cluster regarding using 
RocksDBTimestampedStore when a regular RocksDB store should have been used.

Reviewers: Guozhang Wang , Matthias J. Sax 
, John Roesler 
---
 streams/src/main/java/org/apache/kafka/streams/state/Stores.java | 2 +-
 streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java | 4 +++-
 2 files changed, 4 insertions(+), 2 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java 
b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java
index b672f3d..91c895f 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java
@@ -84,7 +84,7 @@ public class Stores {
  */
 public static KeyValueBytesStoreSupplier persistentKeyValueStore(final 
String name) {
 Objects.requireNonNull(name, "name cannot be null");
-return new RocksDbKeyValueBytesStoreSupplier(name, true);
+return new RocksDbKeyValueBytesStoreSupplier(name, false);
 }
 
 /**
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java 
b/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java
index 1d4a849..9cc1280 100644
--- a/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/state/StoresTest.java
@@ -21,11 +21,13 @@ import 
org.apache.kafka.streams.state.internals.InMemoryKeyValueStore;
 import org.apache.kafka.streams.state.internals.MemoryNavigableLRUCache;
 import org.apache.kafka.streams.state.internals.RocksDBSessionStore;
 import org.apache.kafka.streams.state.internals.RocksDBStore;
+import org.apache.kafka.streams.state.internals.RocksDBTimestampedStore;
 import org.apache.kafka.streams.state.internals.RocksDBWindowStore;
 import org.junit.Test;
 
 import static java.time.Duration.ZERO;
 import static java.time.Duration.ofMillis;
+import static org.hamcrest.CoreMatchers.allOf;
 import static org.hamcrest.CoreMatchers.nullValue;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.core.IsInstanceOf.instanceOf;
@@ -113,7 +115,7 @@ public class StoresTest {
 
 @Test
 public void shouldCreateRocksDbStore() {
-assertThat(Stores.persistentKeyValueStore("store").get(), 
instanceOf(RocksDBStore.class));
+assertThat(Stores.persistentKeyValueStore("store").get(), 
allOf(not(instanceOf(RocksDBTimestampedStore.class)), 
instanceOf(RocksDBStore.class)));
 }
 
 @Test



[kafka] branch 2.1 updated: MINOR: Add all topics created check streams broker bounce test (2.1) (#6242)

2019-02-21 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.1 by this push:
 new 2d94783  MINOR: Add all topics created check streams broker bounce 
test (2.1) (#6242)
2d94783 is described below

commit 2d94783325f8de3bedc20b84e586f0d18b48fee1
Author: Bill Bejeck 
AuthorDate: Thu Feb 21 15:58:13 2019 -0500

MINOR: Add all topics created check streams broker bounce test (2.1) (#6242)

The StreamsBrokerBounceTest.test_broker_type_bounce experienced what looked 
like a transient failure. After looking over this test and failure, it seems 
like it is vulnerable to timing error that streams will start before the kafka 
service creates all topics.

Reviewers: Guozhang Wang , Matthias J. Sax 
, John Roesler 
---
 .../tests/streams/streams_broker_bounce_test.py| 22 ++
 1 file changed, 22 insertions(+)

diff --git a/tests/kafkatest/tests/streams/streams_broker_bounce_test.py 
b/tests/kafkatest/tests/streams/streams_broker_bounce_test.py
index 8d623eb..7859d69 100644
--- a/tests/kafkatest/tests/streams/streams_broker_bounce_test.py
+++ b/tests/kafkatest/tests/streams/streams_broker_bounce_test.py
@@ -133,6 +133,22 @@ class StreamsBrokerBounceTest(Test):
 for num in range(0, num_failures - 1):
 signal_node(self, self.kafka.nodes[num], sig)
 
+def confirm_topics_on_all_brokers(self, expected_topic_set):
+for node in self.kafka.nodes:
+match_count = 0
+# need to iterate over topic_list_generator as kafka.list_topics()
+# returns a python generator so values are fetched lazily
+# so we can't just compare directly we must iterate over what's 
returned
+topic_list_generator = self.kafka.list_topics("placeholder", node)
+for topic in topic_list_generator:
+if topic in expected_topic_set:
+match_count += 1
+
+if len(expected_topic_set) != match_count:
+return False
+
+return True
+
 
 def setup_system(self, start_processor=True):
 # Setup phase
@@ -141,6 +157,12 @@ class StreamsBrokerBounceTest(Test):
 
 self.kafka = KafkaService(self.test_context, 
num_nodes=self.replication, zk=self.zk, topics=self.topics)
 self.kafka.start()
+
+# allow some time for topics to be created
+wait_until(lambda: 
self.confirm_topics_on_all_brokers(set(self.topics.keys())),
+   timeout_sec=60,
+   err_msg="Broker did not create all topics in 60 seconds ")
+
 # Start test harness
 self.driver = StreamsSmokeTestDriverService(self.test_context, 
self.kafka)
 self.processor1 = StreamsSmokeTestJobRunnerService(self.test_context, 
self.kafka)



[kafka] branch trunk updated: MINOR: Missing punctuation marks in quickstart (#5755)

2019-02-21 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 342c336  MINOR: Missing punctuation marks in quickstart (#5755)
342c336 is described below

commit 342c336bf9706a99c65c7fd42b0f5edeccb8adbe
Author: Vito Jeng 
AuthorDate: Fri Feb 22 08:41:03 2019 +0800

MINOR: Missing punctuation marks in quickstart (#5755)

Minor fix for missing punctuation marks in the quickstart.

Reviewers: John Roesler , Bill Bejeck 
---
 docs/streams/quickstart.html | 8 +---
 1 file changed, 5 insertions(+), 3 deletions(-)

diff --git a/docs/streams/quickstart.html b/docs/streams/quickstart.html
index 9abeed4..717fe46 100644
--- a/docs/streams/quickstart.html
+++ b/docs/streams/quickstart.html
@@ -56,8 +56,10 @@ final Serde<Long> longSerde = Serdes.Long();
 // Construct a `KStream` from the input topic "streams-plaintext-input", where 
message values
 // represent lines of text (for the sake of this example, we ignore whatever 
may be stored
 // in the message keys).
-KStream<String, String> textLines = 
builder.stream("streams-plaintext-input",
-Consumed.with(stringSerde, stringSerde);
+KStream<String, String> textLines = builder.stream(
+  "streams-plaintext-input",
+  Consumed.with(stringSerde, stringSerde)
+);
 
 KTable<String, Long> wordCounts = textLines
 // Split each text line, by whitespace, into words.
@@ -67,7 +69,7 @@ KTable<String, Long> wordCounts = textLines
 .groupBy((key, value) -> value)
 
 // Count the occurrences of each word (message key).
-.count()
+.count();
 
 // Store the running counts as a changelog stream to the output topic.
 wordCounts.toStream().to("streams-wordcount-output", 
Produced.with(Serdes.String(), Serdes.Long()));



[kafka] branch trunk updated: MINOR: Missing punctuation marks in quickstart (#5755)

2019-02-21 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 342c336  MINOR: Missing punctuation marks in quickstart (#5755)
342c336 is described below

commit 342c336bf9706a99c65c7fd42b0f5edeccb8adbe
Author: Vito Jeng 
AuthorDate: Fri Feb 22 08:41:03 2019 +0800

MINOR: Missing punctuation marks in quickstart (#5755)

Minor fix for missing punctuation marks in the quickstart.

Reviewers: John Roesler , Bill Bejeck 
---
 docs/streams/quickstart.html | 8 +---
 1 file changed, 5 insertions(+), 3 deletions(-)

diff --git a/docs/streams/quickstart.html b/docs/streams/quickstart.html
index 9abeed4..717fe46 100644
--- a/docs/streams/quickstart.html
+++ b/docs/streams/quickstart.html
@@ -56,8 +56,10 @@ final Serde<Long> longSerde = Serdes.Long();
 // Construct a `KStream` from the input topic "streams-plaintext-input", where 
message values
 // represent lines of text (for the sake of this example, we ignore whatever 
may be stored
 // in the message keys).
-KStream<String, String> textLines = 
builder.stream("streams-plaintext-input",
-Consumed.with(stringSerde, stringSerde);
+KStream<String, String> textLines = builder.stream(
+  "streams-plaintext-input",
+  Consumed.with(stringSerde, stringSerde)
+);
 
 KTable<String, Long> wordCounts = textLines
 // Split each text line, by whitespace, into words.
@@ -67,7 +69,7 @@ KTable<String, Long> wordCounts = textLines
 .groupBy((key, value) -> value)
 
 // Count the occurrences of each word (message key).
-.count()
+.count();
 
 // Store the running counts as a changelog stream to the output topic.
 wordCounts.toStream().to("streams-wordcount-output", 
Produced.with(Serdes.String(), Serdes.Long()));



[kafka] branch trunk updated: KAFKA-7492 : Updated javadocs for aggregate and reduce methods returning null behavior. (#6285)

2019-02-22 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 93ba962  KAFKA-7492 : Updated javadocs for aggregate and reduce 
methods returning null behavior. (#6285)
93ba962 is described below

commit 93ba9621fe0ebe2945fe5d14a3c94abc5cffd7b4
Author: asutosh936 
AuthorDate: Fri Feb 22 11:07:30 2019 -0600

KAFKA-7492 : Updated javadocs for aggregate and reduce methods returning 
null behavior. (#6285)

This is an update to the existing javadocs for KGroupedStream class.

Guozhang Wang , Matthias J. Sax ,  
John Roesler , Bill Bejeck 
---
 .../org/apache/kafka/streams/kstream/KGroupedStream.java | 16 
 1 file changed, 12 insertions(+), 4 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java 
b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java
index 05e4ac9..121d0a4 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java
@@ -146,7 +146,9 @@ public interface KGroupedStream {
  *
  * @param reducer   a {@link Reducer} that computes a new aggregate 
result. Cannot be {@code null}.
  * @return a {@link KTable} that contains "update" records with unmodified 
keys, and values that represent the
- * latest (rolling) aggregate for each key
+ * latest (rolling) aggregate for each key. If the reduce function returns 
{@code null}, it is then interpreted as
+ * deletion for the key, and future messages of the same key coming from 
upstream operators
+ * will be handled as newly initialized value.
  */
 KTable reduce(final Reducer reducer);
 
@@ -208,7 +210,9 @@ public interface KGroupedStream {
  * @param reducer   a {@link Reducer} that computes a new aggregate 
result. Cannot be {@code null}.
  * @param materialized  an instance of {@link Materialized} used to 
materialize a state store. Cannot be {@code null}.
  * @return a {@link KTable} that contains "update" records with unmodified 
keys, and values that represent the
- * latest (rolling) aggregate for each key
+ * latest (rolling) aggregate for each key. If the reduce function returns 
{@code null}, it is then interpreted as
+ * deletion for the key, and future messages of the same key coming from 
upstream operators
+ * will be handled as newly initialized value.
  */
 KTable reduce(final Reducer reducer,
 final Materialized> 
materialized);
@@ -251,7 +255,9 @@ public interface KGroupedStream {
  * @param aggregatoran {@link Aggregator} that computes a new 
aggregate result
  * @param   the value type of the resulting {@link KTable}
  * @return a {@link KTable} that contains "update" records with unmodified 
keys, and values that represent the
- * latest (rolling) aggregate for each key
+ * latest (rolling) aggregate for each key. If the aggregate function 
returns {@code null}, it is then interpreted as
+ * deletion for the key, and future messages of the same key coming from 
upstream operators
+ * will be handled as newly initialized value.
  */
  KTable aggregate(final Initializer initializer,
  final Aggregator 
aggregator);
@@ -308,7 +314,9 @@ public interface KGroupedStream {
  * @param materialized  an instance of {@link Materialized} used to 
materialize a state store. Cannot be {@code null}.
  * @param   the value type of the resulting {@link KTable}
  * @return a {@link KTable} that contains "update" records with unmodified 
keys, and values that represent the
- * latest (rolling) aggregate for each key
+ * latest (rolling) aggregate for each key. If the aggregate function 
returns {@code null}, it is then interpreted as
+ * deletion for the key, and future messages of the same key coming from 
upstream operators
+ * will be handled as newly initialized value.
  */
  KTable aggregate(final Initializer initializer,
  final Aggregator 
aggregator,



[kafka] branch trunk updated: KAFKA-7492 : Updated javadocs for aggregate and reduce methods returning null behavior. (#6285)

2019-02-22 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 93ba962  KAFKA-7492 : Updated javadocs for aggregate and reduce 
methods returning null behavior. (#6285)
93ba962 is described below

commit 93ba9621fe0ebe2945fe5d14a3c94abc5cffd7b4
Author: asutosh936 
AuthorDate: Fri Feb 22 11:07:30 2019 -0600

KAFKA-7492 : Updated javadocs for aggregate and reduce methods returning 
null behavior. (#6285)

This is an update to the existing javadocs for KGroupedStream class.

Guozhang Wang , Matthias J. Sax ,  
John Roesler , Bill Bejeck 
---
 .../org/apache/kafka/streams/kstream/KGroupedStream.java | 16 
 1 file changed, 12 insertions(+), 4 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java 
b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java
index 05e4ac9..121d0a4 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java
@@ -146,7 +146,9 @@ public interface KGroupedStream {
  *
  * @param reducer   a {@link Reducer} that computes a new aggregate 
result. Cannot be {@code null}.
  * @return a {@link KTable} that contains "update" records with unmodified 
keys, and values that represent the
- * latest (rolling) aggregate for each key
+ * latest (rolling) aggregate for each key. If the reduce function returns 
{@code null}, it is then interpreted as
+ * deletion for the key, and future messages of the same key coming from 
upstream operators
+ * will be handled as newly initialized value.
  */
 KTable reduce(final Reducer reducer);
 
@@ -208,7 +210,9 @@ public interface KGroupedStream {
  * @param reducer   a {@link Reducer} that computes a new aggregate 
result. Cannot be {@code null}.
  * @param materialized  an instance of {@link Materialized} used to 
materialize a state store. Cannot be {@code null}.
  * @return a {@link KTable} that contains "update" records with unmodified 
keys, and values that represent the
- * latest (rolling) aggregate for each key
+ * latest (rolling) aggregate for each key. If the reduce function returns 
{@code null}, it is then interpreted as
+ * deletion for the key, and future messages of the same key coming from 
upstream operators
+ * will be handled as newly initialized value.
  */
 KTable reduce(final Reducer reducer,
 final Materialized> 
materialized);
@@ -251,7 +255,9 @@ public interface KGroupedStream {
  * @param aggregatoran {@link Aggregator} that computes a new 
aggregate result
  * @param   the value type of the resulting {@link KTable}
  * @return a {@link KTable} that contains "update" records with unmodified 
keys, and values that represent the
- * latest (rolling) aggregate for each key
+ * latest (rolling) aggregate for each key. If the aggregate function 
returns {@code null}, it is then interpreted as
+ * deletion for the key, and future messages of the same key coming from 
upstream operators
+ * will be handled as newly initialized value.
  */
  KTable aggregate(final Initializer initializer,
  final Aggregator 
aggregator);
@@ -308,7 +314,9 @@ public interface KGroupedStream {
  * @param materialized  an instance of {@link Materialized} used to 
materialize a state store. Cannot be {@code null}.
  * @param   the value type of the resulting {@link KTable}
  * @return a {@link KTable} that contains "update" records with unmodified 
keys, and values that represent the
- * latest (rolling) aggregate for each key
+ * latest (rolling) aggregate for each key. If the aggregate function 
returns {@code null}, it is then interpreted as
+ * deletion for the key, and future messages of the same key coming from 
upstream operators
+ * will be handled as newly initialized value.
  */
  KTable aggregate(final Initializer initializer,
  final Aggregator 
aggregator,



[kafka] branch trunk updated: MINOR: Update docs to say 2.2 (#6315)

2019-02-23 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new b3689b0  MINOR: Update docs to say 2.2 (#6315)
b3689b0 is described below

commit b3689b0c0e09851f41dd7013e57178760de70743
Author: Matthias J. Sax 
AuthorDate: Sat Feb 23 06:03:00 2019 -0800

MINOR: Update docs to say 2.2 (#6315)

Update docs to say 2.2

Reviewers: Jason Gustafson , Bill Bejeck 

---
 docs/documentation.html | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/documentation.html b/docs/documentation.html
index 4e2abc3..83c029b 100644
--- a/docs/documentation.html
+++ b/docs/documentation.html
@@ -26,8 +26,8 @@


 Documentation
-Kafka 2.1 Documentation
-Prior releases: 0.7.x, 0.8.0, 0.8.1.X, 0.8.2.X, 0.9.0.X, 0.10.0.X, 0.10.1.X, 0.10.2.X, 0.11.0.X, 1.0.X, Kafka 2.2 Documentation
+Prior releases: 0.7.x, 0.8.0, 0.8.1.X, 0.8.2.X, 0.9.0.X, 0.10.0.X, 0.10.1.X, 0.10.2.X, 0.11.0.X, 1.0.X, 
 



[kafka] branch trunk updated: MINOR: Update docs to say 2.2 (#6315)

2019-02-23 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new b3689b0  MINOR: Update docs to say 2.2 (#6315)
b3689b0 is described below

commit b3689b0c0e09851f41dd7013e57178760de70743
Author: Matthias J. Sax 
AuthorDate: Sat Feb 23 06:03:00 2019 -0800

MINOR: Update docs to say 2.2 (#6315)

Update docs to say 2.2

Reviewers: Jason Gustafson , Bill Bejeck 

---
 docs/documentation.html | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/documentation.html b/docs/documentation.html
index 4e2abc3..83c029b 100644
--- a/docs/documentation.html
+++ b/docs/documentation.html
@@ -26,8 +26,8 @@


 Documentation
-Kafka 2.1 Documentation
-Prior releases: 0.7.x, 0.8.0, 0.8.1.X, 0.8.2.X, 0.9.0.X, 0.10.0.X, 0.10.1.X, 0.10.2.X, 0.11.0.X, 1.0.X, Kafka 2.2 Documentation
+Prior releases: 0.7.x, 0.8.0, 0.8.1.X, 0.8.2.X, 0.9.0.X, 0.10.0.X, 0.10.1.X, 0.10.2.X, 0.11.0.X, 1.0.X, 
 



[kafka] branch 2.2 updated: MINOR: Update docs to say 2.2 (#6315)

2019-02-23 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new 9bd3563  MINOR: Update docs to say 2.2 (#6315)
9bd3563 is described below

commit 9bd35634ef6bcef464c498bda89176d4558581ba
Author: Matthias J. Sax 
AuthorDate: Sat Feb 23 06:03:00 2019 -0800

MINOR: Update docs to say 2.2 (#6315)

Update docs to say 2.2

Reviewers: Jason Gustafson , Bill Bejeck 

---
 docs/documentation.html | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/documentation.html b/docs/documentation.html
index 4e2abc3..83c029b 100644
--- a/docs/documentation.html
+++ b/docs/documentation.html
@@ -26,8 +26,8 @@


 Documentation
-Kafka 2.1 Documentation
-Prior releases: 0.7.x, 0.8.0, 0.8.1.X, 0.8.2.X, 0.9.0.X, 0.10.0.X, 0.10.1.X, 0.10.2.X, 0.11.0.X, 1.0.X, Kafka 2.2 Documentation
+Prior releases: 0.7.x, 0.8.0, 0.8.1.X, 0.8.2.X, 0.9.0.X, 0.10.0.X, 0.10.1.X, 0.10.2.X, 0.11.0.X, 1.0.X, 
 



[kafka] branch trunk updated: [TRIVIAL] Remove unused StreamsGraphNode#repartitionRequired (#6227)

2019-02-26 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new dc91ce5  [TRIVIAL] Remove unused StreamsGraphNode#repartitionRequired 
(#6227)
dc91ce5 is described below

commit dc91ce58af941b822fbad3b560c35774130848f0
Author: Lee Dongjin 
AuthorDate: Wed Feb 27 00:22:25 2019 +0900

[TRIVIAL] Remove unused StreamsGraphNode#repartitionRequired (#6227)

I found this defect while inspecting [KAFKA-7293: Merge followed by 
groupByKey/join might violate 
co-partitioning](https://issues.apache.org/jira/browse/KAFKA-7293); This flag 
is never used now. Instead, `KStreamImpl#repartitionRequired` is now covering 
its functionality.

Reviewers: Matthias J. Sax , Bill Bejeck 

---
 .../internals/GroupedStreamAggregateBuilder.java   |  3 +-
 .../kstream/internals/InternalStreamsBuilder.java  |  2 +-
 .../kstream/internals/KGroupedTableImpl.java   |  3 +-
 .../streams/kstream/internals/KStreamImpl.java | 41 +-
 .../streams/kstream/internals/KTableImpl.java  |  8 ++---
 .../internals/graph/BaseJoinProcessorNode.java |  3 +-
 .../internals/graph/BaseRepartitionNode.java   |  2 +-
 .../internals/graph/ProcessorGraphNode.java| 14 ++--
 .../kstream/internals/graph/StateStoreNode.java|  2 +-
 .../internals/graph/StatefulProcessorNode.java | 14 +++-
 .../kstream/internals/graph/StreamSinkNode.java|  3 +-
 .../kstream/internals/graph/StreamSourceNode.java  |  4 +--
 .../internals/graph/StreamTableJoinNode.java   |  3 +-
 .../kstream/internals/graph/StreamsGraphNode.java  | 10 +-
 .../internals/graph/TableProcessorNode.java|  2 +-
 .../kstream/internals/AbstractStreamTest.java  |  3 +-
 .../internals/graph/GraphGraceSearchUtilTest.java  | 24 +
 17 files changed, 46 insertions(+), 95 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java
index 46546f4..cd5155f 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java
@@ -102,8 +102,7 @@ class GroupedStreamAggregateBuilder {
 new StatefulProcessorNode<>(
 aggFunctionName,
 new ProcessorParameters<>(aggregateSupplier, aggFunctionName),
-storeBuilder,
-repartitionRequired
+storeBuilder
 );
 
 builder.addGraphNode(parentNode, statefulProcessorNode);
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 e0983eb..c06b988 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
@@ -65,7 +65,7 @@ public class InternalStreamsBuilder implements 
InternalNameProvider {
 private static final String TOPOLOGY_ROOT = "root";
 private static final Logger LOG = 
LoggerFactory.getLogger(InternalStreamsBuilder.class);
 
-protected final StreamsGraphNode root = new 
StreamsGraphNode(TOPOLOGY_ROOT, false) {
+protected final StreamsGraphNode root = new 
StreamsGraphNode(TOPOLOGY_ROOT) {
 @Override
 public void writeToTopology(final InternalTopologyBuilder 
topologyBuilder) {
 // no-op for root node
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java
index 4675f56..56be0f6 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java
@@ -88,8 +88,7 @@ public class KGroupedTableImpl extends 
AbstractStream implements KGr
 final StatefulProcessorNode statefulProcessorNode = new 
StatefulProcessorNode<>(
 funcName,
 new ProcessorParameters<>(aggregateSupplier, funcName),
-new KeyValueStoreMaterializer<>(materialized).materialize(),
-false
+new KeyValueStoreMaterializer<>(materialized).materialize()
 );
 
 // now the repartition node must be the parent of the 
StateProcessorNode
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
 
b/streams/src/main/java/org/apache/kafka/st

[kafka] branch trunk updated: [TRIVIAL] Remove unused StreamsGraphNode#repartitionRequired (#6227)

2019-02-26 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new dc91ce5  [TRIVIAL] Remove unused StreamsGraphNode#repartitionRequired 
(#6227)
dc91ce5 is described below

commit dc91ce58af941b822fbad3b560c35774130848f0
Author: Lee Dongjin 
AuthorDate: Wed Feb 27 00:22:25 2019 +0900

[TRIVIAL] Remove unused StreamsGraphNode#repartitionRequired (#6227)

I found this defect while inspecting [KAFKA-7293: Merge followed by 
groupByKey/join might violate 
co-partitioning](https://issues.apache.org/jira/browse/KAFKA-7293); This flag 
is never used now. Instead, `KStreamImpl#repartitionRequired` is now covering 
its functionality.

Reviewers: Matthias J. Sax , Bill Bejeck 

---
 .../internals/GroupedStreamAggregateBuilder.java   |  3 +-
 .../kstream/internals/InternalStreamsBuilder.java  |  2 +-
 .../kstream/internals/KGroupedTableImpl.java   |  3 +-
 .../streams/kstream/internals/KStreamImpl.java | 41 +-
 .../streams/kstream/internals/KTableImpl.java  |  8 ++---
 .../internals/graph/BaseJoinProcessorNode.java |  3 +-
 .../internals/graph/BaseRepartitionNode.java   |  2 +-
 .../internals/graph/ProcessorGraphNode.java| 14 ++--
 .../kstream/internals/graph/StateStoreNode.java|  2 +-
 .../internals/graph/StatefulProcessorNode.java | 14 +++-
 .../kstream/internals/graph/StreamSinkNode.java|  3 +-
 .../kstream/internals/graph/StreamSourceNode.java  |  4 +--
 .../internals/graph/StreamTableJoinNode.java   |  3 +-
 .../kstream/internals/graph/StreamsGraphNode.java  | 10 +-
 .../internals/graph/TableProcessorNode.java|  2 +-
 .../kstream/internals/AbstractStreamTest.java  |  3 +-
 .../internals/graph/GraphGraceSearchUtilTest.java  | 24 +
 17 files changed, 46 insertions(+), 95 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java
index 46546f4..cd5155f 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/GroupedStreamAggregateBuilder.java
@@ -102,8 +102,7 @@ class GroupedStreamAggregateBuilder {
 new StatefulProcessorNode<>(
 aggFunctionName,
 new ProcessorParameters<>(aggregateSupplier, aggFunctionName),
-storeBuilder,
-repartitionRequired
+storeBuilder
 );
 
 builder.addGraphNode(parentNode, statefulProcessorNode);
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 e0983eb..c06b988 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
@@ -65,7 +65,7 @@ public class InternalStreamsBuilder implements 
InternalNameProvider {
 private static final String TOPOLOGY_ROOT = "root";
 private static final Logger LOG = 
LoggerFactory.getLogger(InternalStreamsBuilder.class);
 
-protected final StreamsGraphNode root = new 
StreamsGraphNode(TOPOLOGY_ROOT, false) {
+protected final StreamsGraphNode root = new 
StreamsGraphNode(TOPOLOGY_ROOT) {
 @Override
 public void writeToTopology(final InternalTopologyBuilder 
topologyBuilder) {
 // no-op for root node
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java
index 4675f56..56be0f6 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java
@@ -88,8 +88,7 @@ public class KGroupedTableImpl extends 
AbstractStream implements KGr
 final StatefulProcessorNode statefulProcessorNode = new 
StatefulProcessorNode<>(
 funcName,
 new ProcessorParameters<>(aggregateSupplier, funcName),
-new KeyValueStoreMaterializer<>(materialized).materialize(),
-false
+new KeyValueStoreMaterializer<>(materialized).materialize()
 );
 
 // now the repartition node must be the parent of the 
StateProcessorNode
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
 
b/streams/src/main/java/org/apache/kafka/st

[kafka-site] branch asf-site updated: Add keys for bbejeck (#188)

2019-02-26 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/kafka-site.git


The following commit(s) were added to refs/heads/asf-site by this push:
 new 175fb4c  Add keys for bbejeck (#188)
175fb4c is described below

commit 175fb4c914d7e7aa59af69abe8f5c3638d70abf4
Author: Bill Bejeck 
AuthorDate: Tue Feb 26 12:15:11 2019 -0500

Add keys for bbejeck (#188)

Adding key for bbejeck

Reviewers: Matthias J. Sax 
---
 KEYS | 60 
 1 file changed, 60 insertions(+)

diff --git a/KEYS b/KEYS
index 1a4fb49..e88e5a9 100644
--- a/KEYS
+++ b/KEYS
@@ -1070,3 +1070,63 @@ 
gm+FU3WdD9oqC3/dDKZ7pFizUhijUZo+nfedhj+7+ZGPFrPX11TS+N3p2E863DGG
 gGDok4uiZw4HdrV8d3/JvahBXfQWH12KeoFmZi/pY/XiPZszEyPrdRVrNbRag/0=
 =V7qY
 -END PGP PUBLIC KEY BLOCK-
+pub   rsa4096 2019-02-23 [SC]
+  1BE3 316E 762D 17FF 4D2D  55B6 7D19 0980 87E7 3B62
+uid   [ultimate] Bill Bejeck (CODE SIGNING KEY) 
+sub   rsa4096 2019-02-23 [E]
+
+-BEGIN PGP PUBLIC KEY BLOCK-
+
+mQINBFxxo3wBEACe7CV/KvBUovUy+TvtmrWCL+9t3dAnteudZiFMOARXn7WyUmww
+Yc0gsl+hYRoiebBxHqgJ3ER233DdPrZjabx7Q4gyDciIaX/Xgp53S+7KZe+IT1rr
+I+fuYeh7Xlvdetna38jwseXvVKXa3WJYUvdu92iFj5Pcaw290mXfE3i+Xm0idlcm
+tw6YImJecS/7JqyF5DpFLSgHEtqhGTR9cdP6zcOoj80RzNza0e8BlN+eOBNpr4Vj
+9rIfcw15IdhdAokYEQ6ek8dLFWP+XHZst81JeaokTzB4TrtIhyYudLwTnW2x6JOY
+QTmni4g0ysex7BKnTKbzCJYYlocTGbtlzE7ZE02KQOUmWplOoYDLPFb80DtdQwSM
+c7PEg0z9+zWgXiCeMbfU5Ji7KE7o74k5CASnVHJ7PBvTYapIOsmYzmBkjqxeFyMu
+5PBud5SH7nVnIC1X3iWJhJKBt9l/D6AbKf09GxpY0MJg5xe0aZBninKaozJf56Fb
+UaHJouHi5tfg16cTC8EvAsKCAPlDiYvqLq4dan7XiIIpe5FwBL/4kJkyeq/zbgBP
+JYPsKB7K0Z8LInuT3YMVOpnPS2jQz6HEoxBjMD+COmeTqy/eBJDLSFLnnD0ZXGus
+gWgoSs5ujcZIrJnpQpDpxFmlpmXrFkRb8AWgoQXfgzQy4x9BsBY/afaTawARAQAB
+tDJCaWxsIEJlamVjayAoQ09ERSBTSUdOSU5HIEtFWSkgPGJiZWplY2tAZ21haWwu
+Y29tPokCTgQTAQoAOBYhBBvjMW52LRf/TS1Vtn0ZCYCH5ztiBQJccaN8AhsDBQsJ
+CAcDBRUKCQgLBRYCAwEAAh4BAheAAAoJEH0ZCYCH5zti8m4P/3xip32e2d1E3fPa
+NB8FWaJlRhSFJCXAE5kxWGHmb0IYEg8HnYzMkX2Cri8pb3rIfZzIoBai9B4u7ADv
+S91zaLFqurD5e6UrmuJOGvHyDKemMzR29qdARDSKsP2T/5ugyb1n/YmBdV8ZZcnw
+sJQUkyZSgTFPatVLqynzrF3H4iExCIJi+5tm9tiuwIyInGSi9qK6cBzmplOz3EtA
+4q0lbJTqk9LQyQMgk2f5XBoftL++d+LPcilJr8EsKd6FB0svNGovXJr80csVI6gs
+dKi0TSeI94dVSLhpSqbWVBqdgo/rPdXAzzURm05vLkvtQoDWcfyYayOC8q3DmajI
+WoXts2myPMtXl4SfsrtlYz9xG6cybW2sKtAQSpp0P7T2V1B/+ATz2hKRnAn5FIUg
+7ehfqIoXqQEzwBeX3yDcb99Bj9svxGYInsKwHyqUe9H4piuSq96ynGm57nmuYIxY
+9iXGhAdCu26MwIS5Hmk/t89IWv4PmDjmtqe2ySN57BcRd1I9BlSbJGGQQrRoBmTY
+PkgvfV58jrUJq+jM3fKWWxSvrtFJ6OMv60ZaPNOnTmYYmye0tEAEp2RXD1JKnj4F
+ndrR859PBRnejfWOwLe0++cfq1pCttsjS2Mrbrz7m7rKKJwa48rkXC+Z0qr2BIpu
+Ak3rZ/NZtKje1b4PK7KLtvJuH5XuuQINBFxxo3wBEADFdsFBeWFODMHTon5k++GK
+o3EjQdNONCt8zzW3udzVpg6sJgTbuBh2+QRpjs6d4yVnfDmwq72FVAoiqGr4BflT
+KBpFyE0G5JFzLPJ++uc2iWRK0Wc+MNhRKOttvuAqzZZTuehZ8K+P4j2H3tOR0Y2S
+gL1jCiFgOQRhoKlrtAFs42acgivQijDrtULKuUpeVp6AXUHkuCQjVbHAhbwm5phw
+LDcJXhzBh8gQ3mFk59u35GBLoseiSucgcsVKvj5i0Um59JoAi7jkJ6dR54o/uxgg
+rpuOWngxljPUAqPXo0u2+1WJBfkgnqksx2K60IIlUpBuyriyU7OxTiBcoQd6NX+D
+LlkByHsbdUi4GnQo8NkAmIbTw3Vei74QnMgsMy/iFmsYmFZyB1hKK5M4KlTvB11h
+Gq61vpNVLrvTH+83oAdfm6i88wJGsXBXgL/0oU7vxxnNIBlnggEYt/RY93HjokBf
+KZHEAviAwqHQ2PefLI2EMcYJu1j+dvgjtW9FJMyDHZPiHL4ZUU6uuDhkc5kXP/ki
+G08oaNXIrQqMUBEhGkA2JzjksjJ0yRHm0oVXwSNqecTbPo0XvoSA0lh8EPyOiPFH
+c5kFOrOwB2+ZQXb61LxAHA2xEeCud3qOQBa8zCDYfireaBSicTFRtnBENbNoVFO+
+WRDETUqaGpjHLaV+i7RqjwARAQABiQI2BBgBCgAgFiEEG+MxbnYtF/9NLVW2fRkJ
+gIfnO2IFAlxxo3wCGwwACgkQfRkJgIfnO2IcSA//Waj2L1gTbbWxK3iZdZj1PiUh
+1+2j/OeqH7xZ6W/CXsQVyu4hm8pyHPVuOfji4Es9RX23IJq5aycEMTePNoX4Z1IN
+UiFHdBj2kQkEkMkWx9f4rVGB0PahT2DjbVd7mc92m4U1IrBL/ii9Y/X7IhOHuMBq
+jXmN4aw/jpQPx/1A7wzwBUkl5zCjWJz0oCu2xypFGpYqdvc6s2lXBeEoC3VDIiNb
+z+p4rkh1ZRNN/t8tof3xjevTb+gShwC845d2LlyKz2TM2L3ps7BmvjXnMf3GxtvA
+8FQN218OZVy2K8JErK4nGOsTgwWucDL4EPaxVgBD2TqHn3bTohljNH/9ElR1KIEe
+k2tQIDmXZCMIcQGxjgVEv5nQ4v0JlF3L1kua56M4lP3Nw1DvEBK4n5LULBYSsnta
++BiIUO31gajwnuqCAk2PnLPwonXrLknq5hpjqXNQCbAsCov7STLrN/ItlqJjjXW4
+Y3fZTNOhZePnHUuUHxB8xlx2iXERMtawSyKyDr9t3GU4rCqfMdF1824/mCyqH/ME
+mMUourTyq0HdNTn0IgSLLTG6+Z5gU/lzS82BCzFX1lQ0ubbRPOjVbaJEkZCjGjqY
+HA2n6MDyGfIZA1X3CCN+AoSINMz5a3gC2n14q44RZJUSVgEzQ72ZR+gCVlcFV+Zy
+Y+Zqob82+zQ+EY/PtiM=
+=v46J
+-END PGP PUBLIC KEY BLOCK-
+
+
+



[kafka] branch trunk updated: KAFKA-7918: Inline generic parameters Pt. II: RocksDB Bytes Store and Memory LRU Caches (#6327)

2019-02-27 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new a8f2307  KAFKA-7918: Inline generic parameters Pt. II: RocksDB Bytes 
Store and Memory LRU Caches (#6327)
a8f2307 is described below

commit a8f2307164ce0f1f47c458eee8f54173f7218a16
Author: A. Sophie Blee-Goldman 
AuthorDate: Wed Feb 27 07:08:08 2019 -0800

KAFKA-7918: Inline generic parameters Pt. II: RocksDB Bytes Store and 
Memory LRU Caches (#6327)

Second PR in series to inline the generic parameters of the following bytes 
stores

Reviewers: Matthias J. Sax , Bill Bejeck 

---
 .../org/apache/kafka/streams/state/Stores.java |  3 +-
 .../internals/ChangeLoggingKeyValueBytesStore.java |  2 +-
 .../streams/state/internals/MemoryLRUCache.java| 76 +++---
 .../state/internals/MemoryNavigableLRUCache.java   | 39 +--
 .../state/internals/RocksDBSessionStore.java   | 68 +++
 .../state/internals/RocksDBWindowStore.java| 53 ++-
 .../RocksDbSessionBytesStoreSupplier.java  |  3 +-
 .../internals/RocksDbWindowBytesStoreSupplier.java | 11 ++--
 .../internals/WindowStoreIteratorWrapper.java  | 47 -
 .../internals/WrappedSessionStoreIterator.java | 17 ++---
 .../state/internals/CachingSessionStoreTest.java   |  2 +-
 .../state/internals/CachingWindowStoreTest.java|  4 +-
 .../state/internals/RocksDBSessionStoreTest.java   | 35 --
 .../state/internals/RocksDBWindowStoreTest.java| 18 -
 14 files changed, 126 insertions(+), 252 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java 
b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java
index 46a9d45..113e531 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java
@@ -18,7 +18,6 @@ package org.apache.kafka.streams.state;
 
 import org.apache.kafka.common.annotation.InterfaceStability;
 import org.apache.kafka.common.serialization.Serde;
-import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.common.utils.Time;
 import org.apache.kafka.streams.internals.ApiUtils;
@@ -134,7 +133,7 @@ public class Stores {
 
 @Override
 public KeyValueStore get() {
-return new MemoryNavigableLRUCache<>(name, maxCacheSize, 
Serdes.Bytes(), Serdes.ByteArray());
+return new MemoryNavigableLRUCache(name, maxCacheSize);
 }
 
 @Override
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java
index 7567e78..aa931bf 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java
@@ -44,7 +44,7 @@ public class ChangeLoggingKeyValueBytesStore extends 
WrappedStateStore) 
wrapped()).setWhenEldestRemoved((key, value) -> {
+((MemoryLRUCache) wrapped()).setWhenEldestRemoved((key, value) -> {
 // pass null to indicate removal
 changeLogger.logChange(key, null);
 });
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java
index f0c3c8c..d69df13 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java
@@ -16,14 +16,12 @@
  */
 package org.apache.kafka.streams.state.internals;
 
-import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.processor.StateStore;
-import org.apache.kafka.streams.processor.internals.ProcessorStateManager;
 import org.apache.kafka.streams.state.KeyValueIterator;
 import org.apache.kafka.streams.state.KeyValueStore;
-import org.apache.kafka.streams.state.StateSerdes;
 
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -32,46 +30,31 @@ import java.util.Objects;
 
 /**
  * An in-memory LRU cache store based on HashSet and HashMap.
- *
- *  * Note that the use of array-typed keys is discouraged because they result 
in incorrect ordering behavior.
- * If you intend to work on byte arrays as key, for example, you may want to 
wrap them with the {@code Bytes} class,
- * i.e. use {@code RocksDBStore} rather

[kafka] branch trunk updated: KAFKA-8011: Fix for race condition causing concurrent modification exception (#6338)

2019-02-28 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 45adc80  KAFKA-8011: Fix for race condition causing concurrent 
modification exception (#6338)
45adc80 is described below

commit 45adc80366e6e61187484d7e88467df05af2470f
Author: Bill Bejeck 
AuthorDate: Thu Feb 28 18:54:25 2019 -0500

KAFKA-8011: Fix for race condition causing concurrent modification 
exception (#6338)

In the RegexSourceIntegrationTest#testRegexMatchesTopicsAWhenCreated() and 
RegexSourceIntegrationTest#testRegexMatchesTopicsAWhenDeleted() a race 
condition exists where the ConsumerRebalanceListener in the test modifies the 
list of subscribed topics when the condition for the test success is comparing 
the same array instance against expected values.

This PR should fix this race condition by using a CopyOnWriteArrayList 
which guarantees safe traversal of the list even when a concurrent modification 
is happening.

Reviewers: Guozhang Wang 
---
 .../apache/kafka/streams/integration/RegexSourceIntegrationTest.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git 
a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
index 264cd35..1cb9e0c 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
@@ -60,6 +60,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.regex.Pattern;
 
@@ -152,7 +153,7 @@ public class RegexSourceIntegrationTest {
 final KStream pattern1Stream = 
builder.stream(Pattern.compile("TEST-TOPIC-\\d"));
 
 pattern1Stream.to(DEFAULT_OUTPUT_TOPIC, Produced.with(stringSerde, 
stringSerde));
-final List assignedTopics = new ArrayList<>();
+final List assignedTopics = new CopyOnWriteArrayList<>();
 streams = new KafkaStreams(builder.build(), streamsConfiguration, new 
DefaultKafkaClientSupplier() {
 @Override
 public Consumer getConsumer(final Map config) {
@@ -190,7 +191,7 @@ public class RegexSourceIntegrationTest {
 
 pattern1Stream.to(DEFAULT_OUTPUT_TOPIC, Produced.with(stringSerde, 
stringSerde));
 
-final List assignedTopics = new ArrayList<>();
+final List assignedTopics = new CopyOnWriteArrayList<>();
 streams = new KafkaStreams(builder.build(), streamsConfiguration, new 
DefaultKafkaClientSupplier() {
 @Override
 public Consumer getConsumer(final Map config) {



[kafka] branch 2.2 updated: KAFKA-8011: Fix for race condition causing concurrent modification exception (#6338)

2019-02-28 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new 1b522a1  KAFKA-8011: Fix for race condition causing concurrent 
modification exception (#6338)
1b522a1 is described below

commit 1b522a1d739e7bcafd0c134f31ff6268f0437f25
Author: Bill Bejeck 
AuthorDate: Thu Feb 28 18:54:25 2019 -0500

KAFKA-8011: Fix for race condition causing concurrent modification 
exception (#6338)

In the RegexSourceIntegrationTest#testRegexMatchesTopicsAWhenCreated() and 
RegexSourceIntegrationTest#testRegexMatchesTopicsAWhenDeleted() a race 
condition exists where the ConsumerRebalanceListener in the test modifies the 
list of subscribed topics when the condition for the test success is comparing 
the same array instance against expected values.

This PR should fix this race condition by using a CopyOnWriteArrayList 
which guarantees safe traversal of the list even when a concurrent modification 
is happening.

Reviewers: Guozhang Wang 
---
 .../apache/kafka/streams/integration/RegexSourceIntegrationTest.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git 
a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
index 2873593..fc7a14c 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
@@ -60,6 +60,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.regex.Pattern;
 
@@ -152,7 +153,7 @@ public class RegexSourceIntegrationTest {
 final KStream pattern1Stream = 
builder.stream(Pattern.compile("TEST-TOPIC-\\d"));
 
 pattern1Stream.to(DEFAULT_OUTPUT_TOPIC, Produced.with(stringSerde, 
stringSerde));
-final List assignedTopics = new ArrayList<>();
+final List assignedTopics = new CopyOnWriteArrayList<>();
 streams = new KafkaStreams(builder.build(), streamsConfiguration, new 
DefaultKafkaClientSupplier() {
 @Override
 public Consumer getConsumer(final Map config) {
@@ -190,7 +191,7 @@ public class RegexSourceIntegrationTest {
 
 pattern1Stream.to(DEFAULT_OUTPUT_TOPIC, Produced.with(stringSerde, 
stringSerde));
 
-final List assignedTopics = new ArrayList<>();
+final List assignedTopics = new CopyOnWriteArrayList<>();
 streams = new KafkaStreams(builder.build(), streamsConfiguration, new 
DefaultKafkaClientSupplier() {
 @Override
 public Consumer getConsumer(final Map config) {



[kafka] branch 2.1 updated: KAFKA-8011: Fix for race condition causing concurrent modification exception (#6338)

2019-02-28 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.1 by this push:
 new ce9f522  KAFKA-8011: Fix for race condition causing concurrent 
modification exception (#6338)
ce9f522 is described below

commit ce9f522400e9eba5a36b49a725bc28ebaeaf33c8
Author: Bill Bejeck 
AuthorDate: Thu Feb 28 18:54:25 2019 -0500

KAFKA-8011: Fix for race condition causing concurrent modification 
exception (#6338)

In the RegexSourceIntegrationTest#testRegexMatchesTopicsAWhenCreated() and 
RegexSourceIntegrationTest#testRegexMatchesTopicsAWhenDeleted() a race 
condition exists where the ConsumerRebalanceListener in the test modifies the 
list of subscribed topics when the condition for the test success is comparing 
the same array instance against expected values.

This PR should fix this race condition by using a CopyOnWriteArrayList 
which guarantees safe traversal of the list even when a concurrent modification 
is happening.

Reviewers: Guozhang Wang 
---
 .../apache/kafka/streams/integration/RegexSourceIntegrationTest.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git 
a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
index be87eb2..a3ab289 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
@@ -60,6 +60,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.regex.Pattern;
 
@@ -153,7 +154,7 @@ public class RegexSourceIntegrationTest {
 final KStream pattern1Stream = 
builder.stream(Pattern.compile("TEST-TOPIC-\\d"));
 
 pattern1Stream.to(DEFAULT_OUTPUT_TOPIC, Produced.with(stringSerde, 
stringSerde));
-final List assignedTopics = new ArrayList<>();
+final List assignedTopics = new CopyOnWriteArrayList<>();
 streams = new KafkaStreams(builder.build(), streamsConfiguration, new 
DefaultKafkaClientSupplier() {
 @Override
 public Consumer getConsumer(final Map config) {
@@ -202,7 +203,7 @@ public class RegexSourceIntegrationTest {
 
 pattern1Stream.to(DEFAULT_OUTPUT_TOPIC, Produced.with(stringSerde, 
stringSerde));
 
-final List assignedTopics = new ArrayList<>();
+final List assignedTopics = new CopyOnWriteArrayList<>();
 streams = new KafkaStreams(builder.build(), streamsConfiguration, new 
DefaultKafkaClientSupplier() {
 @Override
 public Consumer getConsumer(final Map config) {



[kafka] branch 2.0 updated: KAFKA-8011: Fix for race condition causing concurrent modification exception (#6338)

2019-02-28 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.0
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.0 by this push:
 new b0dd4d5  KAFKA-8011: Fix for race condition causing concurrent 
modification exception (#6338)
b0dd4d5 is described below

commit b0dd4d50b4fe97365db2475d559dd3f1c6eb271a
Author: Bill Bejeck 
AuthorDate: Thu Feb 28 18:54:25 2019 -0500

KAFKA-8011: Fix for race condition causing concurrent modification 
exception (#6338)

In the RegexSourceIntegrationTest#testRegexMatchesTopicsAWhenCreated() and 
RegexSourceIntegrationTest#testRegexMatchesTopicsAWhenDeleted() a race 
condition exists where the ConsumerRebalanceListener in the test modifies the 
list of subscribed topics when the condition for the test success is comparing 
the same array instance against expected values.

This PR should fix this race condition by using a CopyOnWriteArrayList 
which guarantees safe traversal of the list even when a concurrent modification 
is happening.

Reviewers: Guozhang Wang 
---
 .../apache/kafka/streams/integration/RegexSourceIntegrationTest.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git 
a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
index d035586..0c6f21e 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
@@ -58,6 +58,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.regex.Pattern;
 
@@ -146,7 +147,7 @@ public class RegexSourceIntegrationTest {
 final KStream pattern1Stream = 
builder.stream(Pattern.compile("TEST-TOPIC-\\d"));
 
 pattern1Stream.to(DEFAULT_OUTPUT_TOPIC, Produced.with(stringSerde, 
stringSerde));
-final List assignedTopics = new ArrayList<>();
+final List assignedTopics = new CopyOnWriteArrayList<>();
 streams = new KafkaStreams(builder.build(), streamsConfiguration, new 
DefaultKafkaClientSupplier() {
 @Override
 public Consumer getConsumer(final Map config) {
@@ -195,7 +196,7 @@ public class RegexSourceIntegrationTest {
 
 pattern1Stream.to(DEFAULT_OUTPUT_TOPIC, Produced.with(stringSerde, 
stringSerde));
 
-final List assignedTopics = new ArrayList<>();
+final List assignedTopics = new CopyOnWriteArrayList<>();
 streams = new KafkaStreams(builder.build(), streamsConfiguration, new 
DefaultKafkaClientSupplier() {
 @Override
 public Consumer getConsumer(final Map config) {



[kafka] branch 1.1 updated: Fixing merge conflicts from cherry pick

2019-02-28 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 1.1
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/1.1 by this push:
 new c1b3846  Fixing merge conflicts from cherry pick
c1b3846 is described below

commit c1b38463122a83606bcb955bfd6f12bc8963bae0
Author: Bill Bejeck 
AuthorDate: Thu Feb 28 18:54:25 2019 -0500

Fixing merge conflicts from cherry pick
---
 .../apache/kafka/streams/integration/RegexSourceIntegrationTest.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git 
a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
index 1da4c58..5f0a107 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
@@ -57,6 +57,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.regex.Pattern;
 
 import static org.hamcrest.CoreMatchers.equalTo;
@@ -144,7 +145,7 @@ public class RegexSourceIntegrationTest {
 final KStream pattern1Stream = 
builder.stream(Pattern.compile("TEST-TOPIC-\\d"));
 
 pattern1Stream.to(stringSerde, stringSerde, DEFAULT_OUTPUT_TOPIC);
-final List assignedTopics = new ArrayList<>();
+final List assignedTopics = new CopyOnWriteArrayList<>();
 streams = new KafkaStreams(builder.build(), streamsConfig, new 
DefaultKafkaClientSupplier() {
 @Override
 public Consumer getConsumer(final Map config) {
@@ -195,7 +196,7 @@ public class RegexSourceIntegrationTest {
 
 pattern1Stream.to(stringSerde, stringSerde, DEFAULT_OUTPUT_TOPIC);
 
-final List assignedTopics = new ArrayList<>();
+final List assignedTopics = new CopyOnWriteArrayList<>();
 streams = new KafkaStreams(builder.build(), streamsConfig, new 
DefaultKafkaClientSupplier() {
 @Override
 public Consumer getConsumer(final Map config) {



[kafka] branch 1.0 updated: KAFKA-8011: Fix for race condition causing concurrent modification exception (#6338)

2019-02-28 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 1.0
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/1.0 by this push:
 new 7ef21e7  KAFKA-8011: Fix for race condition causing concurrent 
modification exception (#6338)
7ef21e7 is described below

commit 7ef21e7fab2ea8c93fce04ee6112655c561d3f71
Author: Bill Bejeck 
AuthorDate: Thu Feb 28 18:54:25 2019 -0500

KAFKA-8011: Fix for race condition causing concurrent modification 
exception (#6338)
---
 .../apache/kafka/streams/integration/RegexSourceIntegrationTest.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git 
a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
index 1da4c58..5f0a107 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
@@ -57,6 +57,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.regex.Pattern;
 
 import static org.hamcrest.CoreMatchers.equalTo;
@@ -144,7 +145,7 @@ public class RegexSourceIntegrationTest {
 final KStream pattern1Stream = 
builder.stream(Pattern.compile("TEST-TOPIC-\\d"));
 
 pattern1Stream.to(stringSerde, stringSerde, DEFAULT_OUTPUT_TOPIC);
-final List assignedTopics = new ArrayList<>();
+final List assignedTopics = new CopyOnWriteArrayList<>();
 streams = new KafkaStreams(builder.build(), streamsConfig, new 
DefaultKafkaClientSupplier() {
 @Override
 public Consumer getConsumer(final Map config) {
@@ -195,7 +196,7 @@ public class RegexSourceIntegrationTest {
 
 pattern1Stream.to(stringSerde, stringSerde, DEFAULT_OUTPUT_TOPIC);
 
-final List assignedTopics = new ArrayList<>();
+final List assignedTopics = new CopyOnWriteArrayList<>();
 streams = new KafkaStreams(builder.build(), streamsConfig, new 
DefaultKafkaClientSupplier() {
 @Override
 public Consumer getConsumer(final Map config) {



[kafka] branch 2.1 updated: KAFKA-7895: fix stream-time reckoning for Suppress (2.1) (#6286) (#6325)

2019-03-05 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.1 by this push:
 new 0953cd8  KAFKA-7895: fix stream-time reckoning for Suppress (2.1) 
(#6286) (#6325)
0953cd8 is described below

commit 0953cd8c056e8eb0dd329a4b2d6ebee0d0e8362c
Author: John Roesler 
AuthorDate: Tue Mar 5 13:36:07 2019 -0600

KAFKA-7895: fix stream-time reckoning for Suppress (2.1) (#6286) (#6325)

Even within a Task, different Processors have different perceptions
of time, due to record caching on stores and in suppression itself,
and in general, due to any processor logic that may hold onto
records arbitrarily and emit them later. Thanks to this, we can't rely
on the whole task existing in the same "instant" of stream-time. The
solution is for each processor node that cares about stream-time to
track it independently.

Reviewers: Guozhang Wang , Bill Bejeck 

---
 .../internals/KStreamSessionWindowAggregate.java   |   5 +-
 .../kstream/internals/KStreamWindowAggregate.java  |   5 +-
 .../suppress/KTableSuppressProcessor.java  |  13 ++-
 .../internals/suppress/SuppressedInternal.java |  39 +--
 .../internals/GlobalProcessorContextImpl.java  |   7 +-
 .../internals/InternalProcessorContext.java|   2 -
 .../processor/internals/ProcessorContextImpl.java  |  10 --
 .../streams/processor/internals/RecordQueue.java   |   2 +-
 .../processor/internals/StandbyContextImpl.java|  12 ---
 .../streams/processor/internals/StandbyTask.java   |   8 +-
 .../streams/processor/internals/StreamTask.java|   1 -
 .../processor/internals/TimestampSupplier.java |  21 
 .../internals/RocksDBSegmentedBytesStore.java  |  24 +++--
 .../kafka/streams/state/internals/Segments.java|   8 +-
 ...KStreamSessionWindowAggregateProcessorTest.java |   6 +-
 .../suppress/KTableSuppressProcessorTest.java  |  19 
 .../internals/AbstractProcessorContextTest.java|   5 -
 .../internals/RocksDBSegmentedBytesStoreTest.java  |  20 +++-
 .../state/internals/RocksDBWindowStoreTest.java|   5 +-
 .../streams/state/internals/SegmentsTest.java  |  72 ++---
 .../kafka/streams/tests/SmokeTestClient.java   | 120 +
 .../kafka/streams/tests/SmokeTestDriver.java   |  58 +-
 .../kafka/test/InternalMockProcessorContext.java   |  14 +--
 .../kafka/test/MockInternalProcessorContext.java   |  10 --
 .../apache/kafka/test/NoOpProcessorContext.java|   5 -
 25 files changed, 244 insertions(+), 247 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java
index b89399b..f7802d6 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.kstream.internals;
 
+import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.errors.ProcessorStateException;
@@ -82,6 +83,7 @@ public class KStreamSessionWindowAggregate 
implements KStreamAggProce
 private StreamsMetricsImpl metrics;
 private InternalProcessorContext internalProcessorContext;
 private Sensor lateRecordDropSensor;
+private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
 
 @SuppressWarnings("unchecked")
 @Override
@@ -108,7 +110,8 @@ public class KStreamSessionWindowAggregate 
implements KStreamAggProce
 return;
 }
 
-final long closeTime = internalProcessorContext.streamTime() - 
windows.gracePeriodMs();
+observedStreamTime = Math.max(observedStreamTime, 
context().timestamp());
+final long closeTime = observedStreamTime - 
windows.gracePeriodMs();
 
 final long timestamp = context().timestamp();
 final List, Agg>> merged = new ArrayList<>();
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
index f292515..c5b2483 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.streams.kstream.internals;
 
+import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache

[kafka] branch trunk updated: Moved 'expired-window-' and 'late-' record-drop to StreamsMetricsImpl (#6355)

2019-03-06 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 14d97aa  Moved 'expired-window-' and 'late-' record-drop to 
StreamsMetricsImpl (#6355)
14d97aa is described below

commit 14d97aabad1d5aab4b9407e24ab523157832934b
Author: A. Sophie Blee-Goldman 
AuthorDate: Wed Mar 6 09:17:58 2019 -0800

Moved 'expired-window-' and 'late-' record-drop to StreamsMetricsImpl 
(#6355)

Moved hard-coded 'expired-window-record-drop' and 'late-record-drop' to 
static Strings in StreamsMetricsImpl

Reviewers: Guozhang Wang , Matthias J. Sax 
,  Bill Bejeck 
---
 .../org/apache/kafka/streams/kstream/internals/metrics/Sensors.java  | 5 +++--
 .../streams/processor/internals/metrics/StreamsMetricsImpl.java  | 3 +++
 .../apache/kafka/streams/state/internals/InMemoryWindowStore.java| 5 +++--
 .../kafka/streams/state/internals/RocksDBSegmentedBytesStore.java| 5 +++--
 4 files changed, 12 insertions(+), 6 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/metrics/Sensors.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/metrics/Sensors.java
index 12c4813..4ecaeb4 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/metrics/Sensors.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/metrics/Sensors.java
@@ -29,6 +29,7 @@ import 
org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
+import static 
org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.LATE_RECORD_DROP;
 import static 
org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.PROCESSOR_NODE_ID_TAG;
 import static 
org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.PROCESSOR_NODE_METRICS_GROUP;
 
@@ -40,14 +41,14 @@ public class Sensors {
 final Sensor sensor = metrics.nodeLevelSensor(
 context.taskId().toString(),
 context.currentNode().name(),
-"late-record-drop",
+LATE_RECORD_DROP,
 Sensor.RecordingLevel.INFO
 );
 StreamsMetricsImpl.addInvocationRateAndCount(
 sensor,
 PROCESSOR_NODE_METRICS_GROUP,
 metrics.tagMap("task-id", context.taskId().toString(), 
PROCESSOR_NODE_ID_TAG, context.currentNode().name()),
-"late-record-drop"
+LATE_RECORD_DROP
 );
 return sensor;
 }
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java
index ec35157..0a47fce 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/metrics/StreamsMetricsImpl.java
@@ -55,6 +55,9 @@ public class StreamsMetricsImpl implements StreamsMetrics {
 public static final String PROCESSOR_NODE_METRICS_GROUP = 
"stream-processor-node-metrics";
 public static final String PROCESSOR_NODE_ID_TAG = "processor-node-id";
 
+public static final String EXPIRED_WINDOW_RECORD_DROP = 
"expired-window-record-drop";
+public static final String LATE_RECORD_DROP = "late-record-drop";
+
 public StreamsMetricsImpl(final Metrics metrics, final String threadName) {
 Objects.requireNonNull(metrics, "Metrics cannot be null");
 this.threadName = threadName;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java
index 77820c5..67eec0d 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java
@@ -42,6 +42,7 @@ import java.util.NavigableMap;
 import java.util.NoSuchElementException;
 import java.util.TreeMap;
 
+import static 
org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.EXPIRED_WINDOW_RECORD_DROP;
 import static 
org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.addInvocationRateAndCount;
 import static 
org.apache.kafka.streams.state.internals.WindowKeySchema.extractStoreKeyBytes;
 import static 
org.apache.kafka.streams.state.internals.WindowKeySchema.extractStoreTimestamp;
@@ -95,14 +96,14 @@ public class InMemoryWindowStore implements 
WindowStore {
 expiredRecordSensor = metrics.stor

[kafka] branch trunk updated: Fixed docs regarding caching default (#6375)

2019-03-06 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new e62e230  Fixed docs regarding caching default (#6375)
e62e230 is described below

commit e62e23099b8de2414b2961e7d87be311e06db4e1
Author: A. Sophie Blee-Goldman 
AuthorDate: Wed Mar 6 10:54:36 2019 -0800

Fixed docs regarding caching default (#6375)

The current docs are out of date regarding the default caching state.

Reviewers: Guozhang Wang ,  Bill Bejeck 

---
 docs/streams/developer-guide/memory-mgmt.html | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)

diff --git a/docs/streams/developer-guide/memory-mgmt.html 
b/docs/streams/developer-guide/memory-mgmt.html
index 7ae2060..f21ed34 100644
--- a/docs/streams/developer-guide/memory-mgmt.html
+++ b/docs/streams/developer-guide/memory-mgmt.html
@@ -155,10 +155,9 @@
 This means that all downstream processor nodes can see all records, 
whereas the state stores see a reduced number of records.
 This does not impact correctness of the system, but is a performance 
optimization for the state stores. For example, with the
 Processor API you can store a record in a state store while forwarding 
a different value downstream.
-  Following from the example first shown in section State Stores, to enable caching, you can
-add the withCachingEnabled call (note that caches are 
disabled by default and there is no explicit withDisableCaching
+  Following from the example first shown in section State Stores, to disable caching, you can
+add the withCachingDisabled call (note that caches are 
enabled by default, however there is an explicit withCachingEnabled
 call).
-  Tip: Caches are disabled by default and there is no 
explicit disableCaching call).
   StoreBuilder countStoreBuilder =
   Stores.keyValueStoreBuilder(
 Stores.persistentKeyValueStore("Counts"),



[kafka] branch trunk updated: KAFKA-8007: Avoid copying on fetch in InMemoryWindowStore (#6335)

2019-03-06 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 4788863  KAFKA-8007: Avoid copying on fetch in InMemoryWindowStore 
(#6335)
4788863 is described below

commit 47888630a05260f548988d943a633120845f767d
Author: A. Sophie Blee-Goldman 
AuthorDate: Wed Mar 6 11:02:27 2019 -0800

KAFKA-8007: Avoid copying on fetch in InMemoryWindowStore (#6335)

Rewrote the InMemoryWindowStore implementation by moving the work of a 
fetch to the iterator, and cleaned up the iterators as well.

Reviewers: Guozhang Wang , Bill Bejeck 

---
 .../state/internals/InMemoryWindowStore.java   | 296 -
 .../state/internals/InMemoryWindowStoreTest.java   | 108 +++-
 2 files changed, 284 insertions(+), 120 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java
index 67eec0d..7d1b279 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java
@@ -17,6 +17,11 @@
 package org.apache.kafka.streams.state.internals;
 
 import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.utils.Bytes;
@@ -33,20 +38,15 @@ import org.apache.kafka.streams.state.KeyValueIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.LinkedList;
-import java.util.List;
-import java.util.ListIterator;
 import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NavigableMap;
 import java.util.NoSuchElementException;
-import java.util.TreeMap;
 
 import static 
org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.EXPIRED_WINDOW_RECORD_DROP;
 import static 
org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.addInvocationRateAndCount;
 import static 
org.apache.kafka.streams.state.internals.WindowKeySchema.extractStoreKeyBytes;
 import static 
org.apache.kafka.streams.state.internals.WindowKeySchema.extractStoreTimestamp;
 
+
 public class InMemoryWindowStore implements WindowStore {
 
 private static final Logger LOG = 
LoggerFactory.getLogger(InMemoryWindowStore.class);
@@ -63,22 +63,24 @@ public class InMemoryWindowStore implements 
WindowStore {
 private final long windowSize;
 private final boolean retainDuplicates;
 
-private final NavigableMap> segmentMap;
+private final ConcurrentNavigableMap> segmentMap;
+private final Set openIterators;
 
 private volatile boolean open = false;
 
 InMemoryWindowStore(final String name,
-   final long retentionPeriod,
-   final long windowSize,
-   final boolean retainDuplicates,
-   final String metricScope) {
+final long retentionPeriod,
+final long windowSize,
+final boolean retainDuplicates,
+final String metricScope) {
 this.name = name;
 this.retentionPeriod = retentionPeriod;
 this.windowSize = windowSize;
 this.retainDuplicates = retainDuplicates;
 this.metricScope = metricScope;
 
-this.segmentMap = new TreeMap<>();
+this.openIterators = ConcurrentHashMap.newKeySet();
+this.segmentMap = new ConcurrentSkipListMap<>();
 }
 
 @Override
@@ -132,7 +134,7 @@ public class InMemoryWindowStore implements 
WindowStore {
 LOG.debug("Skipping record for expired segment.");
 } else {
 if (value != null) {
-this.segmentMap.computeIfAbsent(windowStartTimestamp, t -> new 
TreeMap<>());
+this.segmentMap.computeIfAbsent(windowStartTimestamp, t -> new 
ConcurrentSkipListMap<>());
 this.segmentMap.get(windowStartTimestamp).put(keyBytes, value);
 } else {
 this.segmentMap.computeIfPresent(windowStartTimestamp, (t, 
kvMap) -> {
@@ -147,7 +149,11 @@ public class InMemoryWindowStore implements 
WindowStore {
 public byte[] fetch(final Bytes key, final long windowStartTimestamp) {
 removeExpiredSegments();
 
-final NavigableMap kvMap = 
this.segmentMap.get(windowStartTimestamp);
+if (windowStartTimestamp <= this.observedStreamTime

[kafka] branch trunk updated: MINOR: improve JavaDocs about global state stores (#6359)

2019-03-06 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 240d758  MINOR: improve JavaDocs about global state stores (#6359)
240d758 is described below

commit 240d7589d624b72fa95e2f84a84778bc3a127927
Author: Matthias J. Sax 
AuthorDate: Wed Mar 6 18:19:47 2019 -0800

MINOR: improve JavaDocs about global state stores (#6359)

Improve JavaDocs about global state stores.

Reviewers: Guozhang Wang , Sophie Blee-Goldman 
,  Bill Bejeck 
---
 .../org/apache/kafka/streams/StreamsBuilder.java   |   9 ++
 .../org/apache/kafka/streams/kstream/KStream.java  | 142 +++--
 .../kafka/streams/scala/StreamsBuilder.scala   |  10 +-
 .../kafka/streams/scala/kstream/KStream.scala  |  25 ++--
 4 files changed, 110 insertions(+), 76 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java 
b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java
index 1b3b4a2..9e89d7a 100644
--- a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java
+++ b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java
@@ -25,9 +25,12 @@ import org.apache.kafka.streams.kstream.KGroupedTable;
 import org.apache.kafka.streams.kstream.KStream;
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.streams.kstream.Transformer;
+import org.apache.kafka.streams.kstream.ValueTransformer;
 import org.apache.kafka.streams.kstream.internals.ConsumedInternal;
 import org.apache.kafka.streams.kstream.internals.InternalStreamsBuilder;
 import org.apache.kafka.streams.kstream.internals.MaterializedInternal;
+import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorSupplier;
 import org.apache.kafka.streams.processor.StateStore;
 import org.apache.kafka.streams.processor.TimestampExtractor;
@@ -447,6 +450,9 @@ public class StreamsBuilder {
 
 /**
  * Adds a state store to the underlying {@link Topology}.
+ * 
+ * It is required to connect state stores to {@link Processor Processors}, 
{@link Transformer Transformers},
+ * or {@link ValueTransformer ValueTransformers} before they can be used.
  *
  * @param builder the builder used to obtain this state store {@link 
StateStore} instance
  * @return itself
@@ -492,6 +498,9 @@ public class StreamsBuilder {
  * records forwarded from the {@link SourceNode}.
  * This {@link ProcessorNode} should be used to keep the {@link 
StateStore} up-to-date.
  * The default {@link TimestampExtractor} as specified in the {@link 
StreamsConfig config} is used.
+ * 
+ * It is not required to connect a global store to {@link Processor 
Processors}, {@link Transformer Transformers},
+ * or {@link ValueTransformer ValueTransformer}; those have read-only 
access to all global stores by default.
  *
  * @param storeBuilder  user defined {@link StoreBuilder}; can't 
be {@code null}
  * @param topic the topic to source the data from
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java 
b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
index 5138917..df001d0 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
@@ -229,18 +229,19 @@ public interface KStream {
  * and emit a record {@code } for each word.
  * {@code
  * KStream inputStream = builder.stream("topic");
- * KStream outputStream = inputStream.flatMap(new 
KeyValueMapper>> {
- * Iterable> apply(byte[] key, String value) 
{
- * String[] tokens = value.split(" ");
- * List> result = new 
ArrayList<>(tokens.length);
+ * KStream outputStream = inputStream.flatMap(
+ * new KeyValueMapper>> {
+ * Iterable> apply(byte[] key, String 
value) {
+ * String[] tokens = value.split(" ");
+ * List> result = new 
ArrayList<>(tokens.length);
+ *
+ * for(String token : tokens) {
+ * result.add(new KeyValue<>(token, 1));
+ * }
  *
- * for(String token : tokens) {
- * result.add(new KeyValue<>(token, 1));
+ * return result;
  * }
- *
- * return result;
- * }
- * });
+ * });
  * }
  * The provided {@link KeyValueMapper} must return an {@link Iterable} 
(e.g., any {@link java.util.Collection} type)
  * and the return value must not be {@code null}.
@@ -497,7 +498,8 @@ public interface KStream {
 

[kafka] branch trunk updated: Improve API docs of (flatT|t)ransform (#6365)

2019-03-07 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 96c4e32  Improve API docs of (flatT|t)ransform (#6365)
96c4e32 is described below

commit 96c4e323bfc886b654b198896e78a7d21c80e30a
Author: cadonna 
AuthorDate: Thu Mar 7 23:41:57 2019 +0100

Improve API docs of (flatT|t)ransform (#6365)

This commit is a follow-up of pull request #5273

Reviewers: Matthias J. Sax , Bill Bejeck 

---
 .../org/apache/kafka/streams/kstream/KStream.java  | 61 +++---
 1 file changed, 41 insertions(+), 20 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java 
b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
index df001d0..d264390 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
@@ -494,9 +494,9 @@ public interface KStream {
  * A {@link Transformer} (provided by the given {@link 
TransformerSupplier}) is applied to each input record and
  * returns zero or one output record.
  * Thus, an input record {@code } can be transformed into an output 
record {@code }.
- * This is a stateful record-by-record operation (cf. {@link 
#map(KeyValueMapper)}).
- * Furthermore, via {@link 
org.apache.kafka.streams.processor.Punctuator#punctuate(long)}, the processing 
progress
- * can be observed and additional periodic actions can be performed.
+ * This is a stateful record-by-record operation (cf. {@link 
#map(KeyValueMapper) map()}).
+ * Furthermore, via {@link 
org.apache.kafka.streams.processor.Punctuator#punctuate(long) 
Punctuator#punctuate()},
+ * the processing progress can be observed and additional periodic actions 
can be performed.
  * 
  * In order to assign a state, the state must be created and registered 
beforehand (it's not required to connect
  * global state stores; read-only access to global state stores is 
available by default):
@@ -511,11 +511,13 @@ public interface KStream {
  *
  * KStream outputStream = inputStream.transform(new TransformerSupplier() 
{ ... }, "myTransformState");
  * }
- * Within the {@link Transformer}, the state is obtained via the {@link  
ProcessorContext}.
+ * Within the {@link Transformer}, the state is obtained via the {@link 
ProcessorContext}.
  * To trigger periodic actions via {@link 
org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()},
  * a schedule must be registered.
  * The {@link Transformer} must return a {@link KeyValue} type in {@link 
Transformer#transform(Object, Object)
  * transform()} and {@link 
org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}.
+ * The return value of {@link Transformer#transform(Object, Object) 
Transformer#transform()} may be {@code null},
+ * in which case no record is emitted.
  * {@code
  * new TransformerSupplier() {
  * Transformer get() {
@@ -543,19 +545,24 @@ public interface KStream {
  * }
  * }
  * Even if any upstream operation was key-changing, no auto-repartition is 
triggered.
- * If repartitioning is required, a call to {@link #through(String)} 
should be performed before {@code transform()}.
+ * If repartitioning is required, a call to {@link #through(String) 
through()} should be performed before
+ * {@code transform()}.
  * 
  * Transforming records might result in an internal data redistribution if 
a key based operator (like an aggregation
  * or join) is applied to the result {@code KStream}.
- * (cf. {@link #transformValues(ValueTransformerSupplier, String...)})
+ * (cf. {@link #transformValues(ValueTransformerSupplier, String...) 
transformValues()} )
  * 
  * Note that it is possible to emit multiple records for each input record 
by using
- * {@link ProcessorContext#forward(Object, Object) context#forward()} in 
{@link Transformer#transform(K, V)}.
- * However, a mismatch between the types of the emitted records and the 
type of the stream would only be detected
- * at runtime.
- * To ensure type-safety at compile-time, it is recommended to use
- * {@link #flatTransform(TransformerSupplier, String...)} if multiple 
records need to be emitted for each input
- * record.
+ * {@link ProcessorContext#forward(Object, Object) context#forward()} in 
{@link Transformer#transform(Object, Object) Transformer#transform()} and
+ * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) 
Punctuator#punctuate()}.
+ * Be aware that a mismatch between the types of the emitted records and 
the type of the stream would only be
+ * detected at runtime.
+ * To ensure type-safety 

[kafka] branch 2.2 updated: Improve API docs of (flatT|t)ransform (#6365)

2019-03-07 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new 8179e8f  Improve API docs of (flatT|t)ransform (#6365)
8179e8f is described below

commit 8179e8fdc04d8968e2aa11d7959bdbbecea19141
Author: cadonna 
AuthorDate: Thu Mar 7 23:41:57 2019 +0100

Improve API docs of (flatT|t)ransform (#6365)

This commit is a follow-up of pull request #5273

Reviewers: Matthias J. Sax , Bill Bejeck 

---
 .../org/apache/kafka/streams/kstream/KStream.java  | 61 +++---
 1 file changed, 41 insertions(+), 20 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java 
b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
index 5138917..c61b703 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
@@ -493,9 +493,9 @@ public interface KStream {
  * A {@link Transformer} (provided by the given {@link 
TransformerSupplier}) is applied to each input record and
  * returns zero or one output record.
  * Thus, an input record {@code } can be transformed into an output 
record {@code }.
- * This is a stateful record-by-record operation (cf. {@link 
#map(KeyValueMapper)}).
- * Furthermore, via {@link 
org.apache.kafka.streams.processor.Punctuator#punctuate(long)}, the processing 
progress
- * can be observed and additional periodic actions can be performed.
+ * This is a stateful record-by-record operation (cf. {@link 
#map(KeyValueMapper) map()}).
+ * Furthermore, via {@link 
org.apache.kafka.streams.processor.Punctuator#punctuate(long) 
Punctuator#punctuate()},
+ * the processing progress can be observed and additional periodic actions 
can be performed.
  * 
  * In order to assign a state, the state must be created and registered 
beforehand:
  * {@code
@@ -509,11 +509,13 @@ public interface KStream {
  *
  * KStream outputStream = inputStream.transform(new TransformerSupplier() 
{ ... }, "myTransformState");
  * }
- * Within the {@link Transformer}, the state is obtained via the {@link  
ProcessorContext}.
+ * Within the {@link Transformer}, the state is obtained via the {@link 
ProcessorContext}.
  * To trigger periodic actions via {@link 
org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()},
  * a schedule must be registered.
  * The {@link Transformer} must return a {@link KeyValue} type in {@link 
Transformer#transform(Object, Object)
  * transform()} and {@link 
org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}.
+ * The return value of {@link Transformer#transform(Object, Object) 
Transformer#transform()} may be {@code null},
+ * in which case no record is emitted.
  * {@code
  * new TransformerSupplier() {
  * Transformer get() {
@@ -541,19 +543,24 @@ public interface KStream {
  * }
  * }
  * Even if any upstream operation was key-changing, no auto-repartition is 
triggered.
- * If repartitioning is required, a call to {@link #through(String)} 
should be performed before {@code transform()}.
+ * If repartitioning is required, a call to {@link #through(String) 
through()} should be performed before
+ * {@code transform()}.
  * 
  * Transforming records might result in an internal data redistribution if 
a key based operator (like an aggregation
  * or join) is applied to the result {@code KStream}.
- * (cf. {@link #transformValues(ValueTransformerSupplier, String...)})
+ * (cf. {@link #transformValues(ValueTransformerSupplier, String...) 
transformValues()} )
  * 
  * Note that it is possible to emit multiple records for each input record 
by using
- * {@link ProcessorContext#forward(Object, Object) context#forward()} in 
{@link Transformer#transform(K, V)}.
- * However, a mismatch between the types of the emitted records and the 
type of the stream would only be detected
- * at runtime.
- * To ensure type-safety at compile-time, it is recommended to use
- * {@link #flatTransform(TransformerSupplier, String...)} if multiple 
records need to be emitted for each input
- * record.
+ * {@link ProcessorContext#forward(Object, Object) context#forward()} in 
{@link Transformer#transform(Object, Object) Transformer#transform()} and
+ * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) 
Punctuator#punctuate()}.
+ * Be aware that a mismatch between the types of the emitted records and 
the type of the stream would only be
+ * detected at runtime.
+ * To ensure type-safety at compile-time, {@link 
ProcessorContext#forward(Object, Object) context#forward()} should
+ * not be used

[kafka] branch trunk updated: KAFKA-8040: Streams handle initTransactions timeout (#6372)

2019-03-08 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new f716d08  KAFKA-8040: Streams handle initTransactions timeout (#6372)
f716d08 is described below

commit f716d08c9c7f0f3ec7456cb9a27a42104457b867
Author: John Roesler 
AuthorDate: Fri Mar 8 09:29:22 2019 -0600

KAFKA-8040: Streams handle initTransactions timeout (#6372)

As of 2.0, Producer.initTransactions may throw a TimeoutException, which is 
retriable. Streams should retry instead of crashing when we encounter this 
exception

Reviewers: Guozhang Wang , Matthias J. Sax 
,  Bill Bejeck 
---
 .../processor/internals/RecordCollectorImpl.java   |  19 ++-
 .../streams/processor/internals/StreamTask.java|  24 +++-
 .../processor/internals/StreamTaskTest.java| 135 -
 3 files changed, 170 insertions(+), 8 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
index 757a5fb..2e9ead8 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
@@ -218,11 +218,20 @@ public class RecordCollectorImpl implements 
RecordCollector {
 }
 });
 } catch (final TimeoutException e) {
-log.error("Timeout exception caught when sending record to topic 
{}. " +
-"This might happen if the producer cannot send data to the 
Kafka cluster and thus, " +
-"its internal buffer fills up. " +
-"You can increase producer parameter `max.block.ms` to 
increase this timeout.", topic);
-throw new StreamsException(String.format("%sFailed to send record 
to topic %s due to timeout.", logPrefix, topic));
+log.error(
+"Timeout exception caught when sending record to topic {}. " +
+"This might happen if the producer cannot send data to the 
Kafka cluster and thus, " +
+"its internal buffer fills up. " +
+"This can also happen if the broker is slow to respond, if 
the network connection to " +
+"the broker was interrupted, or if similar circumstances 
arise. " +
+"You can increase producer parameter `max.block.ms` to 
increase this timeout.",
+topic,
+e
+);
+throw new StreamsException(
+String.format("%sFailed to send record to topic %s due to 
timeout.", logPrefix, topic),
+e
+);
 } catch (final Exception uncaughtException) {
 if (uncaughtException instanceof KafkaException &&
 uncaughtException.getCause() instanceof 
ProducerFencedException) {
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
index ca33756..65761e7 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
@@ -25,6 +25,7 @@ import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.MetricName;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.ProducerFencedException;
+import org.apache.kafka.common.errors.TimeoutException;
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.metrics.stats.Avg;
 import org.apache.kafka.common.metrics.stats.Count;
@@ -246,7 +247,7 @@ public class StreamTask extends AbstractTask implements 
ProcessorNodePunctuator
 // initialize transactions if eos is turned on, which will block if 
the previous transaction has not
 // completed yet; do not start the first transaction until the 
topology has been initialized later
 if (eosEnabled) {
-this.producer.initTransactions();
+initializeTransactions();
 }
 }
 
@@ -298,7 +299,7 @@ public class StreamTask extends AbstractTask implements 
ProcessorNodePunctuator
 throw new IllegalStateException("Task producer should be 
null.");
 }
 producer = producerSupplier.get();
-producer.initTransactions();
+initializeTransactions();
 recordCollector.init(producer);
 
 if (stateMgr.checkpoint != null) {
@@ -872,4 +873,23 

[kafka] branch 2.1 updated: KAFKA-8040: Streams handle initTransactions timeout (#6372)

2019-03-08 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.1 by this push:
 new b4c675b  KAFKA-8040: Streams handle initTransactions timeout (#6372)
b4c675b is described below

commit b4c675b8e03fec28f1de2ac0ae270fc4388fdf57
Author: John Roesler 
AuthorDate: Fri Mar 8 09:29:22 2019 -0600

KAFKA-8040: Streams handle initTransactions timeout (#6372)

As of 2.0, Producer.initTransactions may throw a TimeoutException, which is 
retriable. Streams should retry instead of crashing when we encounter this 
exception

Reviewers: Guozhang Wang , Matthias J. Sax 
,  Bill Bejeck 
---
 .../processor/internals/RecordCollectorImpl.java   |  19 ++-
 .../streams/processor/internals/StreamTask.java|  24 +++-
 .../processor/internals/StreamTaskTest.java| 135 -
 3 files changed, 170 insertions(+), 8 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
index d3a0030..9f9e100 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
@@ -204,11 +204,20 @@ public class RecordCollectorImpl implements 
RecordCollector {
 }
 });
 } catch (final TimeoutException e) {
-log.error("Timeout exception caught when sending record to topic 
{}. " +
-"This might happen if the producer cannot send data to the 
Kafka cluster and thus, " +
-"its internal buffer fills up. " +
-"You can increase producer parameter `max.block.ms` to 
increase this timeout.", topic);
-throw new StreamsException(String.format("%sFailed to send record 
to topic %s due to timeout.", logPrefix, topic));
+log.error(
+"Timeout exception caught when sending record to topic {}. " +
+"This might happen if the producer cannot send data to the 
Kafka cluster and thus, " +
+"its internal buffer fills up. " +
+"This can also happen if the broker is slow to respond, if 
the network connection to " +
+"the broker was interrupted, or if similar circumstances 
arise. " +
+"You can increase producer parameter `max.block.ms` to 
increase this timeout.",
+topic,
+e
+);
+throw new StreamsException(
+String.format("%sFailed to send record to topic %s due to 
timeout.", logPrefix, topic),
+e
+);
 } catch (final Exception uncaughtException) {
 if (uncaughtException instanceof KafkaException &&
 uncaughtException.getCause() instanceof 
ProducerFencedException) {
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
index c4fecef..53abf82 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
@@ -25,6 +25,7 @@ import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.MetricName;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.ProducerFencedException;
+import org.apache.kafka.common.errors.TimeoutException;
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.metrics.stats.Avg;
 import org.apache.kafka.common.metrics.stats.Count;
@@ -242,7 +243,7 @@ public class StreamTask extends AbstractTask implements 
ProcessorNodePunctuator
 // initialize transactions if eos is turned on, which will block if 
the previous transaction has not
 // completed yet; do not start the first transaction until the 
topology has been initialized later
 if (eosEnabled) {
-this.producer.initTransactions();
+initializeTransactions();
 }
 }
 
@@ -294,7 +295,7 @@ public class StreamTask extends AbstractTask implements 
ProcessorNodePunctuator
 throw new IllegalStateException("Task producer should be 
null.");
 }
 producer = producerSupplier.get();
-producer.initTransactions();
+initializeTransactions();
 recordCollector.init(producer);
 }
 }
@@ -841,4 +842,23 @@ public class StreamTask extends AbstractTask imp

[kafka] branch 2.2 updated: KAFKA-8040: Streams handle initTransactions timeout (#6372)

2019-03-08 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new ebbde63  KAFKA-8040: Streams handle initTransactions timeout (#6372)
ebbde63 is described below

commit ebbde6378c61776b468e2e9843b6b70b30d5688c
Author: John Roesler 
AuthorDate: Fri Mar 8 09:29:22 2019 -0600

KAFKA-8040: Streams handle initTransactions timeout (#6372)

As of 2.0, Producer.initTransactions may throw a TimeoutException, which is 
retriable. Streams should retry instead of crashing when we encounter this 
exception

Reviewers: Guozhang Wang , Matthias J. Sax 
,  Bill Bejeck 
---
 .../processor/internals/RecordCollectorImpl.java   |  19 ++-
 .../streams/processor/internals/StreamTask.java|  24 +++-
 .../processor/internals/StreamTaskTest.java| 135 -
 3 files changed, 170 insertions(+), 8 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
index 757a5fb..2e9ead8 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
@@ -218,11 +218,20 @@ public class RecordCollectorImpl implements 
RecordCollector {
 }
 });
 } catch (final TimeoutException e) {
-log.error("Timeout exception caught when sending record to topic 
{}. " +
-"This might happen if the producer cannot send data to the 
Kafka cluster and thus, " +
-"its internal buffer fills up. " +
-"You can increase producer parameter `max.block.ms` to 
increase this timeout.", topic);
-throw new StreamsException(String.format("%sFailed to send record 
to topic %s due to timeout.", logPrefix, topic));
+log.error(
+"Timeout exception caught when sending record to topic {}. " +
+"This might happen if the producer cannot send data to the 
Kafka cluster and thus, " +
+"its internal buffer fills up. " +
+"This can also happen if the broker is slow to respond, if 
the network connection to " +
+"the broker was interrupted, or if similar circumstances 
arise. " +
+"You can increase producer parameter `max.block.ms` to 
increase this timeout.",
+topic,
+e
+);
+throw new StreamsException(
+String.format("%sFailed to send record to topic %s due to 
timeout.", logPrefix, topic),
+e
+);
 } catch (final Exception uncaughtException) {
 if (uncaughtException instanceof KafkaException &&
 uncaughtException.getCause() instanceof 
ProducerFencedException) {
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
index ca33756..65761e7 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
@@ -25,6 +25,7 @@ import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.MetricName;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.ProducerFencedException;
+import org.apache.kafka.common.errors.TimeoutException;
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.metrics.stats.Avg;
 import org.apache.kafka.common.metrics.stats.Count;
@@ -246,7 +247,7 @@ public class StreamTask extends AbstractTask implements 
ProcessorNodePunctuator
 // initialize transactions if eos is turned on, which will block if 
the previous transaction has not
 // completed yet; do not start the first transaction until the 
topology has been initialized later
 if (eosEnabled) {
-this.producer.initTransactions();
+initializeTransactions();
 }
 }
 
@@ -298,7 +299,7 @@ public class StreamTask extends AbstractTask implements 
ProcessorNodePunctuator
 throw new IllegalStateException("Task producer should be 
null.");
 }
 producer = producerSupplier.get();
-producer.initTransactions();
+initializeTransactions();
 recordCollector.init(producer);
 
 if (stateMgr.checkpoint != null) {
@@ -872,4 +873,23 @@ public class StreamT

[kafka] branch trunk updated: KAFKA-3522: Add TimestampedWindowStore builder/runtime classes (#6173)

2019-03-08 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 04e2061  KAFKA-3522: Add TimestampedWindowStore builder/runtime 
classes (#6173)
04e2061 is described below

commit 04e206154ac614b7d4d34a7a1b6ba2c882f607b9
Author: Matthias J. Sax 
AuthorDate: Fri Mar 8 09:30:00 2019 -0800

KAFKA-3522: Add TimestampedWindowStore builder/runtime classes (#6173)

Add TimestampedWindowStore builder/runtime classes

Reviewers: Guozhang Wang , Matthias J. Sax 
,  John Roesler ,  Bill Bejeck 

---
 .../internals/InternalTopologyBuilder.java |   7 +-
 .../processor/internals/ProcessorContextImpl.java  |  31 +-
 .../state/internals/CachingWindowStore.java|  47 +
 ...ChangeLoggingTimestampedKeyValueBytesStore.java |   1 +
 ... ChangeLoggingTimestampedWindowBytesStore.java} |  12 ++-
 .../internals/ChangeLoggingWindowBytesStore.java   |  48 ++---
 .../internals/MeteredTimestampedWindowStore.java   |  58 +++
 .../state/internals/MeteredWindowStore.java|  20 ++--
 .../internals/TimestampedWindowStoreBuilder.java   |  74 ++
 .../internals/ProcessorContextImplTest.java| 112 +
 ...ngeLoggingTimestampedWindowBytesStoreTest.java} |  61 +++
 .../ChangeLoggingWindowBytesStoreTest.java |  31 +++---
 .../internals/MeteredTimestampWindowStoreTest.java |  92 +
 13 files changed, 486 insertions(+), 108 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
index 0648fec..334adce 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
@@ -30,6 +30,7 @@ import org.apache.kafka.streams.processor.TopicNameExtractor;
 import org.apache.kafka.streams.state.StoreBuilder;
 import org.apache.kafka.streams.state.internals.SessionStoreBuilder;
 import org.apache.kafka.streams.state.internals.WindowStoreBuilder;
+import org.apache.kafka.streams.state.internals.TimestampedWindowStoreBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -140,6 +141,8 @@ public class InternalTopologyBuilder {
 long retentionPeriod() {
 if (builder instanceof WindowStoreBuilder) {
 return ((WindowStoreBuilder) builder).retentionPeriod();
+} else if (builder instanceof TimestampedWindowStoreBuilder) {
+return ((TimestampedWindowStoreBuilder) 
builder).retentionPeriod();
 } else if (builder instanceof SessionStoreBuilder) {
 return ((SessionStoreBuilder) builder).retentionPeriod();
 } else {
@@ -160,7 +163,9 @@ public class InternalTopologyBuilder {
 }
 
 private boolean isWindowStore() {
-return builder instanceof WindowStoreBuilder || builder instanceof 
SessionStoreBuilder;
+return builder instanceof WindowStoreBuilder
+|| builder instanceof TimestampedWindowStoreBuilder
+|| builder instanceof SessionStoreBuilder;
 }
 
 // Apparently Java strips the generics from this method because we're 
using the raw type for builder,
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
index 36a3750..764d50c 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
@@ -32,6 +32,8 @@ import 
org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
 import org.apache.kafka.streams.state.KeyValueIterator;
 import org.apache.kafka.streams.state.KeyValueStore;
 import org.apache.kafka.streams.state.SessionStore;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
 import org.apache.kafka.streams.state.WindowStore;
 import org.apache.kafka.streams.state.WindowStoreIterator;
 import org.apache.kafka.streams.state.internals.ThreadCache;
@@ -84,6 +86,8 @@ public class ProcessorContextImpl extends 
AbstractProcessorContext implements Re
 if (global != null) {
 if (global instanceof KeyValueStore) {
 return new KeyValueStoreReadOnlyDecorator((KeyValueStore) 
global);
+} else if (global instanceof TimestampedWindowStore) {
+return new 
TimestampedWindowStoreReadOnlyDecorator((TimestampedWindow

[kafka] branch trunk updated: MINOR: cleanup deprectaion annotations (#6290)

2019-03-08 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 3619d2f  MINOR: cleanup deprectaion annotations (#6290)
3619d2f is described below

commit 3619d2f383f65108dfd33686119f675aaeea54b7
Author: Matthias J. Sax 
AuthorDate: Fri Mar 8 12:31:34 2019 -0800

MINOR: cleanup deprectaion annotations (#6290)

If deprecated interface methods are inherited, the @Deprication tag should 
be used (instead on suppressing the deprecation warning).

Reviewers:  Guozhang Wang ,  John Roesler 
,  Bill Bejeck 
---
 .../apache/kafka/streams/kstream/JoinWindows.java  | 14 +
 .../kafka/streams/kstream/SessionWindows.java  |  2 -
 .../apache/kafka/streams/kstream/TimeWindows.java  | 14 ++---
 .../kafka/streams/kstream/UnlimitedWindows.java|  9 +--
 .../org/apache/kafka/streams/kstream/Windows.java  |  3 +-
 .../streams/kstream/internals/SessionWindow.java   |  2 -
 .../internals/AbstractProcessorContext.java|  4 --
 .../ForwardingDisabledProcessorContext.java|  6 +-
 .../internals/GlobalProcessorContextImpl.java  |  4 +-
 .../internals/InternalProcessorContext.java|  3 +
 .../processor/internals/ProcessorContextImpl.java  | 36 ++--
 .../processor/internals/StandbyContextImpl.java|  8 +--
 .../org/apache/kafka/streams/state/Stores.java | 14 ++---
 .../apache/kafka/streams/state/WindowStore.java| 15 +++--
 .../state/internals/CachingWindowStore.java|  6 +-
 .../internals/ChangeLoggingWindowBytesStore.java   |  6 +-
 .../internals/CompositeReadOnlyWindowStore.java| 66 ++
 .../state/internals/MeteredWindowStore.java|  6 +-
 .../state/internals/RocksDBWindowStore.java|  6 +-
 .../internals/AbstractProcessorContextTest.java| 20 ---
 .../ForwardingDisabledProcessorContextTest.java|  2 +
 .../internals/GlobalProcessorContextImplTest.java  |  2 +
 .../processor/internals/ProcessorTopologyTest.java |  4 +-
 .../internals/RecordDeserializerTest.java  |  3 +-
 .../kafka/test/InternalMockProcessorContext.java   | 14 ++---
 .../kafka/test/MockInternalProcessorContext.java   |  8 +--
 .../apache/kafka/test/NoOpProcessorContext.java| 22 
 .../streams/processor/MockProcessorContext.java| 16 --
 28 files changed, 165 insertions(+), 150 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java 
b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java
index 219489f..6331877 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java
@@ -92,7 +92,7 @@ public final class JoinWindows extends Windows {
 this.maintainDurationMs = maintainDurationMs;
 }
 
-@SuppressWarnings("deprecation") // removing segments from Windows will 
fix this
+@Deprecated // removing segments from Windows will fix this
 private JoinWindows(final long beforeMs,
 final long afterMs,
 final long graceMs,
@@ -131,7 +131,6 @@ public final class JoinWindows extends Windows {
  * @param timeDifference join window interval
  * @throws IllegalArgumentException if {@code timeDifference} is negative 
or can't be represented as {@code long milliseconds}
  */
-@SuppressWarnings("deprecation")
 public static JoinWindows of(final Duration timeDifference) throws 
IllegalArgumentException {
 final String msgPrefix = 
prepareMillisCheckFailMsgPrefix(timeDifference, "timeDifference");
 return of(ApiUtils.validateMillisecondDuration(timeDifference, 
msgPrefix));
@@ -148,7 +147,6 @@ public final class JoinWindows extends Windows {
  * @throws IllegalArgumentException if the resulting window size is 
negative
  * @deprecated Use {@link #before(Duration)} instead.
  */
-@SuppressWarnings("deprecation") // removing segments from Windows will 
fix this
 @Deprecated
 public JoinWindows before(final long timeDifferenceMs) throws 
IllegalArgumentException {
 return new JoinWindows(timeDifferenceMs, afterMs, graceMs, 
maintainDurationMs, segments);
@@ -164,7 +162,6 @@ public final class JoinWindows extends Windows {
  * @param timeDifference relative window start time
  * @throws IllegalArgumentException if the resulting window size is 
negative or {@code timeDifference} can't be represented as {@code long 
milliseconds}
  */
-@SuppressWarnings("deprecation") // removing segments from Windows will 
fix this
 public JoinWindows before(final Duration timeDifference) throws 
IllegalArgumentException {
 final String msgPrefix = 
prepareMillisCheckFailMsgPref

[kafka] branch 2.1 updated: HOT_FIX: Changes needed to correct cherry-pick (#6414)

2019-03-08 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.1 by this push:
 new 45f5f04  HOT_FIX: Changes needed to correct cherry-pick (#6414)
45f5f04 is described below

commit 45f5f04c721caed78c88d9b3a991dc8c56b16b9d
Author: Bill Bejeck 
AuthorDate: Fri Mar 8 21:34:43 2019 -0500

HOT_FIX: Changes needed to correct cherry-pick (#6414)

Fixes needed to correct changes from cherry-pick to 2.1

Reviewers: Guozhang Wang , Matthias J. Sax 
,  John Roesler 
---
 .../processor/internals/StreamTaskTest.java|  1 +
 .../internals/testutil/LogCaptureAppender.java | 50 ++
 2 files changed, 51 insertions(+)

diff --git 
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
index af64099..fc128a5 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
@@ -73,6 +73,7 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.Collectors;
 
+import static java.util.Arrays.asList;
 import static java.util.Collections.singletonList;
 import static org.apache.kafka.common.utils.Utils.mkEntry;
 import static org.apache.kafka.common.utils.Utils.mkMap;
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogCaptureAppender.java
 
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogCaptureAppender.java
index ffb8799..a1f7b31 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogCaptureAppender.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogCaptureAppender.java
@@ -24,10 +24,36 @@ import org.apache.log4j.spi.LoggingEvent;
 
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Optional;
 
 public class LogCaptureAppender extends AppenderSkeleton {
 private final LinkedList events = new LinkedList<>();
 
+@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+public static class Event {
+private String level;
+private String message;
+private Optional throwableInfo;
+
+Event(final String level, final String message, final Optional 
throwableInfo) {
+this.level = level;
+this.message = message;
+this.throwableInfo = throwableInfo;
+}
+
+public String getLevel() {
+return level;
+}
+
+public String getMessage() {
+return message;
+}
+
+public Optional getThrowableInfo() {
+return throwableInfo;
+}
+}
+
 public static LogCaptureAppender createAndRegister() {
 final LogCaptureAppender logCaptureAppender = new LogCaptureAppender();
 Logger.getRootLogger().addAppender(logCaptureAppender);
@@ -59,6 +85,30 @@ public class LogCaptureAppender extends AppenderSkeleton {
 return result;
 }
 
+public List getEvents() {
+final LinkedList result = new LinkedList<>();
+synchronized (events) {
+for (final LoggingEvent event : events) {
+final String[] throwableStrRep = event.getThrowableStrRep();
+final Optional throwableString;
+if (throwableStrRep == null) {
+throwableString = Optional.empty();
+} else {
+final StringBuilder throwableStringBuilder = new 
StringBuilder();
+
+for (final String s : throwableStrRep) {
+throwableStringBuilder.append(s);
+}
+
+throwableString = 
Optional.of(throwableStringBuilder.toString());
+}
+
+result.add(new Event(event.getLevel().toString(), 
event.getRenderedMessage(), throwableString));
+}
+}
+return result;
+}
+
 @Override
 public void close() {
 



[kafka] branch trunk updated: MINOR: fix Scala compiler warning (#6417)

2019-03-09 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 0ef2b4c  MINOR: fix Scala compiler warning (#6417)
0ef2b4c is described below

commit 0ef2b4cccea4196256ea28b5429ce95e2507928b
Author: Matthias J. Sax 
AuthorDate: Sat Mar 9 05:09:07 2019 -0800

MINOR: fix Scala compiler warning (#6417)

Reviewers: Guozhang Wang ,  Bill Bejeck 

---
 .../src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)

diff --git 
a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
 
b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
index 4a1df92..1fcba48 100644
--- 
a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
+++ 
b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
@@ -27,7 +27,6 @@ import org.apache.kafka.streams.state.StoreBuilder
 import org.apache.kafka.streams.{Topology, StreamsBuilder => StreamsBuilderJ}
 import org.apache.kafka.streams.scala.kstream._
 import ImplicitConversions._
-import org.apache.kafka.streams.errors.TopologyException
 
 import scala.collection.JavaConverters._
 
@@ -163,7 +162,7 @@ class StreamsBuilder(inner: StreamsBuilderJ = new 
StreamsBuilderJ) {
*
* @param builder the builder used to obtain this state store `StateStore` 
instance
* @return the underlying Java abstraction `StreamsBuilder` after adding the 
`StateStore`
-   * @throws TopologyException if state store supplier is already added
+   * @throws org.apache.kafka.streams.errors.TopologyException if state store 
supplier is already added
* @see `org.apache.kafka.streams.StreamsBuilder#addStateStore`
*/
   def addStateStore(builder: StoreBuilder[_ <: StateStore]): StreamsBuilderJ = 
inner.addStateStore(builder)



[kafka] branch 2.0 updated: KAFKA-8040: Streams handle initTransactions timeout (#6416)

2019-03-11 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.0
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.0 by this push:
 new af7fabc  KAFKA-8040: Streams handle initTransactions timeout (#6416)
af7fabc is described below

commit af7fabc5a9302708ea277e4252a7382f19ac9a11
Author: John Roesler 
AuthorDate: Mon Mar 11 17:18:11 2019 -0500

KAFKA-8040: Streams handle initTransactions timeout (#6416)

https://issues.apache.org/jira/browse/KAFKA-7934

Reviewers: Guozhang Wang , Bill Bejeck 

---
 .../processor/internals/RecordCollectorImpl.java   |  19 ++-
 .../streams/processor/internals/StreamTask.java|  24 +++-
 .../processor/internals/StreamTaskTest.java| 157 +++--
 .../internals/testutil/LogCaptureAppender.java |  52 ++-
 4 files changed, 230 insertions(+), 22 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
index 554cc85..e483f58 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java
@@ -204,11 +204,20 @@ public class RecordCollectorImpl implements 
RecordCollector {
 }
 });
 } catch (final TimeoutException e) {
-log.error("Timeout exception caught when sending record to topic 
{}. " +
-"This might happen if the producer cannot send data to the 
Kafka cluster and thus, " +
-"its internal buffer fills up. " +
-"You can increase producer parameter `max.block.ms` to 
increase this timeout.", topic);
-throw new StreamsException(String.format("%sFailed to send record 
to topic %s due to timeout.", logPrefix, topic));
+log.error(
+"Timeout exception caught when sending record to topic {}. " +
+"This might happen if the producer cannot send data to the 
Kafka cluster and thus, " +
+"its internal buffer fills up. " +
+"This can also happen if the broker is slow to respond, if 
the network connection to " +
+"the broker was interrupted, or if similar circumstances 
arise. " +
+"You can increase producer parameter `max.block.ms` to 
increase this timeout.",
+topic,
+e
+);
+throw new StreamsException(
+String.format("%sFailed to send record to topic %s due to 
timeout.", logPrefix, topic),
+e
+);
 } catch (final Exception uncaughtException) {
 if (uncaughtException instanceof KafkaException &&
 uncaughtException.getCause() instanceof 
ProducerFencedException) {
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
index ce8e3c1..a325b24 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
@@ -25,6 +25,7 @@ import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.MetricName;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.ProducerFencedException;
+import org.apache.kafka.common.errors.TimeoutException;
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.metrics.stats.Avg;
 import org.apache.kafka.common.metrics.stats.Count;
@@ -222,7 +223,7 @@ public class StreamTask extends AbstractTask implements 
ProcessorNodePunctuator
 // initialize transactions if eos is turned on, which will block if 
the previous transaction has not
 // completed yet; do not start the first transaction until the 
topology has been initialized later
 if (eosEnabled) {
-this.producer.initTransactions();
+initializeTransactions();
 }
 }
 
@@ -270,7 +271,7 @@ public class StreamTask extends AbstractTask implements 
ProcessorNodePunctuator
 throw new IllegalStateException("Task producer should be 
null.");
 }
 producer = producerSupplier.get();
-producer.initTransactions();
+initializeTransactions();
 recordCollector.init(producer);
 }
 }
@@ -796,4 +797,23 @@ public class StreamTask extends AbstractTask implements 
ProcessorNodePunctuator
 Producer getProducer

[kafka] branch trunk updated: KAFKA-3522: add missing guards for TimestampedXxxStore (#6356)

2019-03-12 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new ab7ea07  KAFKA-3522: add missing guards for TimestampedXxxStore (#6356)
ab7ea07 is described below

commit ab7ea07f5e57ec405dc7fddce95de7c639a2fd6e
Author: Matthias J. Sax 
AuthorDate: Tue Mar 12 09:28:14 2019 -0700

KAFKA-3522: add missing guards for TimestampedXxxStore (#6356)

Reviewers: John Roesler , Bill Bejeck 
---
 .../processor/internals/ProcessorContextImpl.java  |  27 +-
 .../internals/GlobalProcessorContextImplTest.java  | 106 ++---
 .../internals/ProcessorContextImplTest.java|  95 ++
 3 files changed, 215 insertions(+), 13 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
index c10ea09..5f32a3b 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
@@ -32,6 +32,7 @@ import 
org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
 import org.apache.kafka.streams.state.KeyValueIterator;
 import org.apache.kafka.streams.state.KeyValueStore;
 import org.apache.kafka.streams.state.SessionStore;
+import org.apache.kafka.streams.state.TimestampedKeyValueStore;
 import org.apache.kafka.streams.state.TimestampedWindowStore;
 import org.apache.kafka.streams.state.ValueAndTimestamp;
 import org.apache.kafka.streams.state.WindowStore;
@@ -84,7 +85,9 @@ public class ProcessorContextImpl extends 
AbstractProcessorContext implements Re
 
 final StateStore global = stateManager.getGlobalStore(name);
 if (global != null) {
-if (global instanceof KeyValueStore) {
+if (global instanceof TimestampedKeyValueStore) {
+return new 
TimestampedKeyValueStoreReadOnlyDecorator((TimestampedKeyValueStore) global);
+} else if (global instanceof KeyValueStore) {
 return new KeyValueStoreReadOnlyDecorator((KeyValueStore) 
global);
 } else if (global instanceof TimestampedWindowStore) {
 return new 
TimestampedWindowStoreReadOnlyDecorator((TimestampedWindowStore) global);
@@ -108,7 +111,9 @@ public class ProcessorContextImpl extends 
AbstractProcessorContext implements Re
 }
 
 final StateStore store = stateManager.getStore(name);
-if (store instanceof KeyValueStore) {
+if (store instanceof TimestampedKeyValueStore) {
+return new 
TimestampedKeyValueStoreReadWriteDecorator((TimestampedKeyValueStore) store);
+} else if (store instanceof KeyValueStore) {
 return new KeyValueStoreReadWriteDecorator((KeyValueStore) store);
 } else if (store instanceof TimestampedWindowStore) {
 return new 
TimestampedWindowStoreReadWriteDecorator((TimestampedWindowStore) store);
@@ -294,6 +299,15 @@ public class ProcessorContextImpl extends 
AbstractProcessorContext implements Re
 }
 }
 
+private static class TimestampedKeyValueStoreReadOnlyDecorator
+extends KeyValueStoreReadOnlyDecorator>
+implements TimestampedKeyValueStore {
+
+private TimestampedKeyValueStoreReadOnlyDecorator(final 
TimestampedKeyValueStore inner) {
+super(inner);
+}
+}
+
 private static class WindowStoreReadOnlyDecorator
 extends StateStoreReadOnlyDecorator, K, V>
 implements WindowStore {
@@ -484,6 +498,15 @@ public class ProcessorContextImpl extends 
AbstractProcessorContext implements Re
 }
 }
 
+private static class TimestampedKeyValueStoreReadWriteDecorator
+extends KeyValueStoreReadWriteDecorator>
+implements TimestampedKeyValueStore {
+
+private TimestampedKeyValueStoreReadWriteDecorator(final 
TimestampedKeyValueStore inner) {
+super(inner);
+}
+}
+
 static class WindowStoreReadWriteDecorator
 extends StateStoreReadWriteDecorator, K, V>
 implements WindowStore {
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImplTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImplTest.java
index 4153cca..b36557c 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImplTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImplTest.java
@@ -21,6 +21,10 @@ import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.processor.StateSt

[kafka] branch trunk updated: MINOR: Fix JavaDocs warnings (#6435)

2019-03-13 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new a648ef0  MINOR: Fix JavaDocs warnings (#6435)
a648ef0 is described below

commit a648ef0b44e9c3efbee3d66fc7dae85fea24a74b
Author: Matthias J. Sax 
AuthorDate: Wed Mar 13 09:16:48 2019 -0700

MINOR: Fix JavaDocs warnings (#6435)

Fix JavaDocs Warning
Reviewers: uozhang Wang ,  Bill Bejeck 

---
 .../src/main/java/org/apache/kafka/common/config/ConfigDef.java   | 4 ++--
 .../oauthbearer/OAuthBearerExtensionsValidatorCallback.java   | 2 +-
 streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java  | 8 
 streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java | 2 +-
 4 files changed, 8 insertions(+), 8 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java 
b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
index 7669f1d..1259882 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
@@ -53,7 +53,7 @@ import java.util.regex.Pattern;
  * defs.define("config_with_validator", Type.INT, 42, 
Range.atLeast(0), "Configuration with user provided validator.");
  * defs.define("config_with_dependents", Type.INT, 
"Configuration with dependents.", "group", 1, "Config 
With Dependents", 
Arrays.asList("config_with_default","config_with_validator"));
  *
- * Map<String, String> props = new HashMap<>();
+ * Map<String, String> props = new HashMap<>();
  * props.put("config_with_default", "some value");
  * props.put("config_with_dependents", "some other value");
  *
@@ -1133,7 +1133,7 @@ public class ConfigDef {
  * If dynamicUpdateModes is non-empty, a "Dynamic Update 
Mode" column
  * will be included n the table with the value of the update mode. Default
  * mode is "read-only".
- * @param dynamicUpdateModes Config name -> update mode mapping
+ * @param dynamicUpdateModes Config name -> update mode mapping
  */
 public String toHtmlTable(Map dynamicUpdateModes) {
 boolean hasUpdateModes = !dynamicUpdateModes.isEmpty();
diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerExtensionsValidatorCallback.java
 
b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerExtensionsValidatorCallback.java
index 97ac4d9..eab208b 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerExtensionsValidatorCallback.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/OAuthBearerExtensionsValidatorCallback.java
@@ -76,7 +76,7 @@ public class OAuthBearerExtensionsValidatorCallback 
implements Callback {
 }
 
 /**
- * @return An immutable {@link Map} consisting of the name->error messages 
of extensions which failed validation
+ * @return An immutable {@link Map} consisting of the name->error 
messages of extensions which failed validation
  */
 public Map invalidExtensions() {
 return Collections.unmodifiableMap(invalidExtensions);
diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java 
b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
index 2d1aa79..4f15dea 100644
--- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
+++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
@@ -155,23 +155,23 @@ public class KafkaStreams implements AutoCloseable {
  *
  * 
  * +--+
- * +<- | Created (0)  |
+ * +<- | Created (0)  |
  * |   +-++
  * | |
  * | v
  * |   ++--+--+
  * |   | Re-  |
- * +<- | Balancing (1)| >+
+ * +<- | Balancing (1)| >+
  * |   +-+-+--+  |
  * | | ^ |
  * | v | |
  * |   +--+  v
- * |   | Running (2)  | >+
+ * |   | Running (2)  | >+
  * |   +--+---+  |
  * |  |  |
  * |  v  |
  * |   +--+---+ ++---+
- * +-> | Pending  |<--- | Error (5)  |
+ * +-> | Pending  |<--- | Error (5)  |
  * 

[kafka] branch trunk updated: KAFKA-7027: Add an overload build method in scala (#6373)

2019-03-15 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 853f24a  KAFKA-7027: Add an overload build method in scala (#6373)
853f24a is described below

commit 853f24a4a18b26b4e4cc03555673ee951067fa6e
Author: Massimo Siani 
AuthorDate: Fri Mar 15 15:56:48 2019 +0100

KAFKA-7027: Add an overload build method in scala (#6373)

The Java API can pass a Properties object to StreamsBuilder#build, to 
allow, e.g., topology optimization, while the Scala API does not yet. The 
latter only delegates the work to the underlying Java implementation.

Reviewers: John Roesler ,  Bill Bejeck 

---
 .../kafka/streams/scala/StreamsBuilder.scala   |  11 ++
 .../apache/kafka/streams/scala/TopologyTest.scala  | 175 -
 2 files changed, 184 insertions(+), 2 deletions(-)

diff --git 
a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
 
b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
index 1fcba48..9c4e65a 100644
--- 
a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
+++ 
b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
@@ -19,6 +19,7 @@
  */
 package org.apache.kafka.streams.scala
 
+import java.util.Properties
 import java.util.regex.Pattern
 
 import org.apache.kafka.streams.kstream.GlobalKTable
@@ -183,4 +184,14 @@ class StreamsBuilder(inner: StreamsBuilderJ = new 
StreamsBuilderJ) {
 inner.addGlobalStore(storeBuilder, topic, consumed, stateUpdateSupplier)
 
   def build(): Topology = inner.build()
+
+  /**
+   * Returns the `Topology` that represents the specified processing logic and 
accepts
+   * a `Properties` instance used to indicate whether to optimize topology or 
not.
+   *
+   * @param props the `Properties` used for building possibly optimized 
topology
+   * @return the `Topology` that represents the specified processing logic
+   * @see `org.apache.kafka.streams.StreamsBuilder#build`
+   */
+  def build(props: Properties): Topology = inner.build(props)
 }
diff --git 
a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala
 
b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala
index 6035dd0..3917552 100644
--- 
a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala
+++ 
b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala
@@ -19,20 +19,31 @@
  */
 package org.apache.kafka.streams.scala
 
+import java.time.Duration
+import java.util
+import java.util.{Locale, Properties}
 import java.util.regex.Pattern
 
+import org.apache.kafka.common.serialization.{Serdes => SerdesJ}
 import org.apache.kafka.streams.kstream.{
+  Aggregator,
+  ForeachAction,
+  Initializer,
+  JoinWindows,
   KeyValueMapper,
+  Predicate,
   Reducer,
   Transformer,
   TransformerSupplier,
   ValueJoiner,
   ValueMapper,
+  Joined => JoinedJ,
   KGroupedStream => KGroupedStreamJ,
   KStream => KStreamJ,
-  KTable => KTableJ
+  KTable => KTableJ,
+  Materialized => MaterializedJ
 }
-import org.apache.kafka.streams.processor.ProcessorContext
+import org.apache.kafka.streams.processor.{AbstractProcessor, 
ProcessorContext, ProcessorSupplier}
 import org.apache.kafka.streams.scala.ImplicitConversions._
 import org.apache.kafka.streams.scala.Serdes._
 import org.apache.kafka.streams.scala.kstream._
@@ -268,4 +279,164 @@ class TopologyTest extends JUnitSuite {
 // should match
 assertEquals(getTopologyScala, getTopologyJava)
   }
+
+  @Test def shouldBuildIdenticalTopologyInJavaNScalaProperties(): Unit = {
+
+val props = new Properties()
+props.put(StreamsConfig.TOPOLOGY_OPTIMIZATION, StreamsConfig.OPTIMIZE)
+
+val propsNoOptimization = new Properties()
+propsNoOptimization.put(StreamsConfig.TOPOLOGY_OPTIMIZATION, 
StreamsConfig.NO_OPTIMIZATION)
+
+val AGGREGATION_TOPIC = "aggregationTopic"
+val REDUCE_TOPIC = "reduceTopic"
+val JOINED_TOPIC = "joinedTopic"
+
+// build the Scala topology
+def getTopologyScala: StreamsBuilder = {
+
+  val aggregator = (_: String, v: String, agg: Int) => agg + v.length
+  val reducer = (v1: String, v2: String) => v1 + ":" + v2
+  val processorValueCollector: util.List[String] = new 
util.ArrayList[String]
+
+  val builder: StreamsBuilder = new StreamsBuilder
+
+  val sourceStream: KStream[String, String] =
+builder.stream(inputTopic)(Consumed.`with`(Serdes.String, 
Serdes.String))
+
+  val mappedStream: KStream[String, String] =
+sourceStream.map((k: String, v: String) => 
(k.toUpperCase(Locale.getDefault), v))
+  mappedStream
+ 

[kafka] branch 2.2 updated: KAFKA-7027: Add an overload build method in scala (#6373)

2019-03-15 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new 9ec64a8  KAFKA-7027: Add an overload build method in scala (#6373)
9ec64a8 is described below

commit 9ec64a8a838f83bfc5e3011eec29104424e86679
Author: Massimo Siani 
AuthorDate: Fri Mar 15 15:56:48 2019 +0100

KAFKA-7027: Add an overload build method in scala (#6373)

The Java API can pass a Properties object to StreamsBuilder#build, to 
allow, e.g., topology optimization, while the Scala API does not yet. The 
latter only delegates the work to the underlying Java implementation.

Reviewers: John Roesler ,  Bill Bejeck 

---
 .../kafka/streams/scala/StreamsBuilder.scala   |  11 ++
 .../apache/kafka/streams/scala/TopologyTest.scala  | 175 -
 2 files changed, 184 insertions(+), 2 deletions(-)

diff --git 
a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
 
b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
index 8c5a9b3..4941859 100644
--- 
a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
+++ 
b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
@@ -19,6 +19,7 @@
  */
 package org.apache.kafka.streams.scala
 
+import java.util.Properties
 import java.util.regex.Pattern
 
 import org.apache.kafka.streams.kstream.GlobalKTable
@@ -178,4 +179,14 @@ class StreamsBuilder(inner: StreamsBuilderJ = new 
StreamsBuilderJ) {
 inner.addGlobalStore(storeBuilder, topic, consumed, stateUpdateSupplier)
 
   def build(): Topology = inner.build()
+
+  /**
+   * Returns the `Topology` that represents the specified processing logic and 
accepts
+   * a `Properties` instance used to indicate whether to optimize topology or 
not.
+   *
+   * @param props the `Properties` used for building possibly optimized 
topology
+   * @return the `Topology` that represents the specified processing logic
+   * @see `org.apache.kafka.streams.StreamsBuilder#build`
+   */
+  def build(props: Properties): Topology = inner.build(props)
 }
diff --git 
a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala
 
b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala
index a826401..afa4ae6 100644
--- 
a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala
+++ 
b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala
@@ -19,20 +19,31 @@
  */
 package org.apache.kafka.streams.scala
 
+import java.time.Duration
+import java.util
+import java.util.{Locale, Properties}
 import java.util.regex.Pattern
 
+import org.apache.kafka.common.serialization.{Serdes => SerdesJ}
 import org.apache.kafka.streams.kstream.{
+  Aggregator,
+  ForeachAction,
+  Initializer,
+  JoinWindows,
   KeyValueMapper,
+  Predicate,
   Reducer,
   Transformer,
   TransformerSupplier,
   ValueJoiner,
   ValueMapper,
+  Joined => JoinedJ,
   KGroupedStream => KGroupedStreamJ,
   KStream => KStreamJ,
-  KTable => KTableJ
+  KTable => KTableJ,
+  Materialized => MaterializedJ
 }
-import org.apache.kafka.streams.processor.ProcessorContext
+import org.apache.kafka.streams.processor.{AbstractProcessor, 
ProcessorContext, ProcessorSupplier}
 import org.apache.kafka.streams.scala.ImplicitConversions._
 import org.apache.kafka.streams.scala.Serdes._
 import org.apache.kafka.streams.scala.kstream._
@@ -268,4 +279,164 @@ class TopologyTest extends JUnitSuite {
 // should match
 assertEquals(getTopologyScala(), getTopologyJava())
   }
+
+  @Test def shouldBuildIdenticalTopologyInJavaNScalaProperties(): Unit = {
+
+val props = new Properties()
+props.put(StreamsConfig.TOPOLOGY_OPTIMIZATION, StreamsConfig.OPTIMIZE)
+
+val propsNoOptimization = new Properties()
+propsNoOptimization.put(StreamsConfig.TOPOLOGY_OPTIMIZATION, 
StreamsConfig.NO_OPTIMIZATION)
+
+val AGGREGATION_TOPIC = "aggregationTopic"
+val REDUCE_TOPIC = "reduceTopic"
+val JOINED_TOPIC = "joinedTopic"
+
+// build the Scala topology
+def getTopologyScala: StreamsBuilder = {
+
+  val aggregator = (_: String, v: String, agg: Int) => agg + v.length
+  val reducer = (v1: String, v2: String) => v1 + ":" + v2
+  val processorValueCollector: util.List[String] = new 
util.ArrayList[String]
+
+  val builder: StreamsBuilder = new StreamsBuilder
+
+  val sourceStream: KStream[String, String] =
+builder.stream(inputTopic)(Consumed.`with`(Serdes.String, 
Serdes.String))
+
+  val mappedStream: KStream[String, String] =
+sourceStream.map((k: String, v: String) => 
(k.toUpperCase(Locale.getDefault), v))
+  mappedStream
+ 

[kafka] branch 2.1 updated: KAFKA-7027: Add an overload build method in scala (#6373)

2019-03-15 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.1 by this push:
 new f488b77  KAFKA-7027: Add an overload build method in scala (#6373)
f488b77 is described below

commit f488b776422544e43c93f7610294139701210998
Author: Massimo Siani 
AuthorDate: Fri Mar 15 15:56:48 2019 +0100

KAFKA-7027: Add an overload build method in scala (#6373)

The Java API can pass a Properties object to StreamsBuilder#build, to 
allow, e.g., topology optimization, while the Scala API does not yet. The 
latter only delegates the work to the underlying Java implementation.

Reviewers: John Roesler ,  Bill Bejeck 

---
 .../kafka/streams/scala/StreamsBuilder.scala   |  11 ++
 .../apache/kafka/streams/scala/TopologyTest.scala  | 175 -
 2 files changed, 184 insertions(+), 2 deletions(-)

diff --git 
a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
 
b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
index 8c5a9b3..4941859 100644
--- 
a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
+++ 
b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/StreamsBuilder.scala
@@ -19,6 +19,7 @@
  */
 package org.apache.kafka.streams.scala
 
+import java.util.Properties
 import java.util.regex.Pattern
 
 import org.apache.kafka.streams.kstream.GlobalKTable
@@ -178,4 +179,14 @@ class StreamsBuilder(inner: StreamsBuilderJ = new 
StreamsBuilderJ) {
 inner.addGlobalStore(storeBuilder, topic, consumed, stateUpdateSupplier)
 
   def build(): Topology = inner.build()
+
+  /**
+   * Returns the `Topology` that represents the specified processing logic and 
accepts
+   * a `Properties` instance used to indicate whether to optimize topology or 
not.
+   *
+   * @param props the `Properties` used for building possibly optimized 
topology
+   * @return the `Topology` that represents the specified processing logic
+   * @see `org.apache.kafka.streams.StreamsBuilder#build`
+   */
+  def build(props: Properties): Topology = inner.build(props)
 }
diff --git 
a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala
 
b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala
index a826401..afa4ae6 100644
--- 
a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala
+++ 
b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala
@@ -19,20 +19,31 @@
  */
 package org.apache.kafka.streams.scala
 
+import java.time.Duration
+import java.util
+import java.util.{Locale, Properties}
 import java.util.regex.Pattern
 
+import org.apache.kafka.common.serialization.{Serdes => SerdesJ}
 import org.apache.kafka.streams.kstream.{
+  Aggregator,
+  ForeachAction,
+  Initializer,
+  JoinWindows,
   KeyValueMapper,
+  Predicate,
   Reducer,
   Transformer,
   TransformerSupplier,
   ValueJoiner,
   ValueMapper,
+  Joined => JoinedJ,
   KGroupedStream => KGroupedStreamJ,
   KStream => KStreamJ,
-  KTable => KTableJ
+  KTable => KTableJ,
+  Materialized => MaterializedJ
 }
-import org.apache.kafka.streams.processor.ProcessorContext
+import org.apache.kafka.streams.processor.{AbstractProcessor, 
ProcessorContext, ProcessorSupplier}
 import org.apache.kafka.streams.scala.ImplicitConversions._
 import org.apache.kafka.streams.scala.Serdes._
 import org.apache.kafka.streams.scala.kstream._
@@ -268,4 +279,164 @@ class TopologyTest extends JUnitSuite {
 // should match
 assertEquals(getTopologyScala(), getTopologyJava())
   }
+
+  @Test def shouldBuildIdenticalTopologyInJavaNScalaProperties(): Unit = {
+
+val props = new Properties()
+props.put(StreamsConfig.TOPOLOGY_OPTIMIZATION, StreamsConfig.OPTIMIZE)
+
+val propsNoOptimization = new Properties()
+propsNoOptimization.put(StreamsConfig.TOPOLOGY_OPTIMIZATION, 
StreamsConfig.NO_OPTIMIZATION)
+
+val AGGREGATION_TOPIC = "aggregationTopic"
+val REDUCE_TOPIC = "reduceTopic"
+val JOINED_TOPIC = "joinedTopic"
+
+// build the Scala topology
+def getTopologyScala: StreamsBuilder = {
+
+  val aggregator = (_: String, v: String, agg: Int) => agg + v.length
+  val reducer = (v1: String, v2: String) => v1 + ":" + v2
+  val processorValueCollector: util.List[String] = new 
util.ArrayList[String]
+
+  val builder: StreamsBuilder = new StreamsBuilder
+
+  val sourceStream: KStream[String, String] =
+builder.stream(inputTopic)(Consumed.`with`(Serdes.String, 
Serdes.String))
+
+  val mappedStream: KStream[String, String] =
+sourceStream.map((k: String, v: String) => 
(k.toUpperCase(Locale.getDefault), v))
+  mappedStream
+ 

[kafka] branch trunk updated: KAFKA-8062: Do not remore StateListener when shutting down stream thread (#6468)

2019-03-19 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 6d649f5  KAFKA-8062: Do not remore StateListener when shutting down 
stream thread (#6468)
6d649f5 is described below

commit 6d649f503a964ed9612e79ef2d9e55e26240fbc3
Author: Guozhang Wang 
AuthorDate: Tue Mar 19 07:12:49 2019 -0700

KAFKA-8062: Do not remore StateListener when shutting down stream thread 
(#6468)

In a previous commit #6091, we've fixed a couple of edge cases and hence do 
not need to remove state listener anymore (before that we removed the state 
listener intentionally to avoid some race conditions, which has been gone for 
now).

Reviewers: Matthias J. Sax ,   Bill Bejeck 

---
 .../org/apache/kafka/streams/KafkaStreams.java |  4 +-
 .../streams/processor/internals/StreamThread.java  |  1 -
 .../org/apache/kafka/streams/KafkaStreamsTest.java | 72 --
 3 files changed, 69 insertions(+), 8 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java 
b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
index 4f15dea..315a6bb 100644
--- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
+++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
@@ -191,7 +191,7 @@ public class KafkaStreams implements AutoCloseable {
  *   the instance will be in the ERROR state. The user will need to close 
it.
  */
 public enum State {
-CREATED(1, 3), REBALANCING(2, 3, 5), RUNNING(1, 3, 5), 
PENDING_SHUTDOWN(4), NOT_RUNNING, ERROR(3, 5);
+CREATED(1, 3), REBALANCING(2, 3, 5), RUNNING(1, 3, 5), 
PENDING_SHUTDOWN(4), NOT_RUNNING, ERROR(3);
 
 private final Set validTransitions = new HashSet<>();
 
@@ -857,7 +857,6 @@ public class KafkaStreams implements AutoCloseable {
 // notify all the threads to stop; avoid deadlocks by stopping 
any
 // further state reports from the thread since we're shutting 
down
 for (final StreamThread thread : threads) {
-thread.setStateListener(null);
 thread.shutdown();
 }
 
@@ -872,7 +871,6 @@ public class KafkaStreams implements AutoCloseable {
 }
 
 if (globalStreamThread != null) {
-globalStreamThread.setStateListener(null);
 globalStreamThread.shutdown();
 }
 
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
index 71df0f9..1bd09e4 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
@@ -269,7 +269,6 @@ public class StreamThread extends Thread {
 if (streamThread.assignmentErrorCode.get() == 
StreamsPartitionAssignor.Error.INCOMPLETE_SOURCE_TOPIC_METADATA.code()) {
 log.error("Received error code {} - shutdown", 
streamThread.assignmentErrorCode.get());
 streamThread.shutdown();
-streamThread.setStateListener(null);
 return;
 }
 final long start = time.milliseconds();
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java 
b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
index 6b8b5b5..3e55f29 100644
--- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
@@ -140,14 +140,14 @@ public class KafkaStreamsTest {
 }
 
 @Test
-public void testStateCloseAfterCreate() {
+public void stateShouldTransitToNotRunningIfCloseRightAfterCreated() {
 globalStreams.close();
 
 Assert.assertEquals(KafkaStreams.State.NOT_RUNNING, 
globalStreams.state());
 }
 
 @Test
-public void testStateOneThreadDeadButRebalanceFinish() throws 
InterruptedException {
+public void stateShouldTransitToRunningIfNonDeadThreadsBackToRunning() 
throws InterruptedException {
 final StateListenerStub stateListener = new StateListenerStub();
 globalStreams.setStateListener(stateListener);
 
@@ -171,7 +171,7 @@ public class KafkaStreamsTest {
 Assert.assertEquals(3, stateListener.numChanges);
 Assert.assertEquals(KafkaStreams.State.REBALANCING, 
globalStreams.state());
 
-for (final StreamThread thread: globalStreams.threads) {
+for (final StreamThread thread : globalStreams.threads) {
 thread.stateListener().onChange(
 thread,
 StreamThread.State.PARTITIONS_

[kafka] branch 2.2 updated: KAFKA-8062: Do not remore StateListener when shutting down stream thread (#6468)

2019-03-19 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new 8e734fc  KAFKA-8062: Do not remore StateListener when shutting down 
stream thread (#6468)
8e734fc is described below

commit 8e734fcb5c3eb232db08f1986f3613799fa57fb2
Author: Guozhang Wang 
AuthorDate: Tue Mar 19 07:12:49 2019 -0700

KAFKA-8062: Do not remore StateListener when shutting down stream thread 
(#6468)

In a previous commit #6091, we've fixed a couple of edge cases and hence do 
not need to remove state listener anymore (before that we removed the state 
listener intentionally to avoid some race conditions, which has been gone for 
now).

Reviewers: Matthias J. Sax ,   Bill Bejeck 

---
 .../org/apache/kafka/streams/KafkaStreams.java |  4 +-
 .../streams/processor/internals/StreamThread.java  |  1 -
 .../org/apache/kafka/streams/KafkaStreamsTest.java | 72 --
 3 files changed, 69 insertions(+), 8 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java 
b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
index 2d1aa79..3c8ee16 100644
--- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
+++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
@@ -191,7 +191,7 @@ public class KafkaStreams implements AutoCloseable {
  *   the instance will be in the ERROR state. The user will need to close 
it.
  */
 public enum State {
-CREATED(1, 3), REBALANCING(2, 3, 5), RUNNING(1, 3, 5), 
PENDING_SHUTDOWN(4), NOT_RUNNING, ERROR(3, 5);
+CREATED(1, 3), REBALANCING(2, 3, 5), RUNNING(1, 3, 5), 
PENDING_SHUTDOWN(4), NOT_RUNNING, ERROR(3);
 
 private final Set validTransitions = new HashSet<>();
 
@@ -857,7 +857,6 @@ public class KafkaStreams implements AutoCloseable {
 // notify all the threads to stop; avoid deadlocks by stopping 
any
 // further state reports from the thread since we're shutting 
down
 for (final StreamThread thread : threads) {
-thread.setStateListener(null);
 thread.shutdown();
 }
 
@@ -872,7 +871,6 @@ public class KafkaStreams implements AutoCloseable {
 }
 
 if (globalStreamThread != null) {
-globalStreamThread.setStateListener(null);
 globalStreamThread.shutdown();
 }
 
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
index 71df0f9..1bd09e4 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
@@ -269,7 +269,6 @@ public class StreamThread extends Thread {
 if (streamThread.assignmentErrorCode.get() == 
StreamsPartitionAssignor.Error.INCOMPLETE_SOURCE_TOPIC_METADATA.code()) {
 log.error("Received error code {} - shutdown", 
streamThread.assignmentErrorCode.get());
 streamThread.shutdown();
-streamThread.setStateListener(null);
 return;
 }
 final long start = time.milliseconds();
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java 
b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
index 6b8b5b5..3e55f29 100644
--- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
@@ -140,14 +140,14 @@ public class KafkaStreamsTest {
 }
 
 @Test
-public void testStateCloseAfterCreate() {
+public void stateShouldTransitToNotRunningIfCloseRightAfterCreated() {
 globalStreams.close();
 
 Assert.assertEquals(KafkaStreams.State.NOT_RUNNING, 
globalStreams.state());
 }
 
 @Test
-public void testStateOneThreadDeadButRebalanceFinish() throws 
InterruptedException {
+public void stateShouldTransitToRunningIfNonDeadThreadsBackToRunning() 
throws InterruptedException {
 final StateListenerStub stateListener = new StateListenerStub();
 globalStreams.setStateListener(stateListener);
 
@@ -171,7 +171,7 @@ public class KafkaStreamsTest {
 Assert.assertEquals(3, stateListener.numChanges);
 Assert.assertEquals(KafkaStreams.State.REBALANCING, 
globalStreams.state());
 
-for (final StreamThread thread: globalStreams.threads) {
+for (final StreamThread thread : globalStreams.threads) {
 thread.stateListener().onChange(
 thread,
 StreamThread.State.PARTITIONS_

[kafka] branch trunk updated: KAFKA-8094: Iterating over cache with get(key) is inefficient (#6433)

2019-03-19 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 538bd7e  KAFKA-8094: Iterating over cache with get(key) is inefficient 
(#6433)
538bd7e is described below

commit 538bd7eddf13897245524f015e3207affb03fcdc
Author: A. Sophie Blee-Goldman 
AuthorDate: Tue Mar 19 08:51:10 2019 -0700

KAFKA-8094: Iterating over cache with get(key) is inefficient (#6433)

Use concurrent data structure for the underlying cache in NamedCache, and 
iterate over it with subMap instead of many calls to get()

Reviewers: Guozhang Wang , Bill Bejeck 

---
 .../kafka/streams/state/internals/NamedCache.java  | 18 ++-
 .../kafka/streams/state/internals/ThreadCache.java | 24 +--
 .../streams/state/internals/NamedCacheTest.java| 27 --
 .../streams/state/internals/ThreadCacheTest.java   |  3 ++-
 4 files changed, 21 insertions(+), 51 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java
index 3ce7cbe..0201f20 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java
@@ -16,6 +16,8 @@
  */
 package org.apache.kafka.streams.state.internals;
 
+import java.util.NavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
 import org.apache.kafka.common.MetricName;
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.metrics.stats.Avg;
@@ -33,13 +35,11 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
 
 class NamedCache {
 private static final Logger log = 
LoggerFactory.getLogger(NamedCache.class);
 private final String name;
-private final TreeMap cache = new TreeMap<>();
+private final NavigableMap cache = new 
ConcurrentSkipListMap<>();
 private final Set dirtyKeys = new LinkedHashSet<>();
 private ThreadCache.DirtyEntryFlushListener listener;
 private LRUNode tail;
@@ -266,16 +266,12 @@ class NamedCache {
 return cache.size();
 }
 
-synchronized Iterator keyRange(final Bytes from, final Bytes to) {
-return keySetIterator(cache.navigableKeySet().subSet(from, true, to, 
true));
+synchronized Iterator> subMapIterator(final 
Bytes from, final Bytes to) {
+return cache.subMap(from, true, to, true).entrySet().iterator();
 }
 
-private Iterator keySetIterator(final Set keySet) {
-return new TreeSet<>(keySet).iterator();
-}
-
-synchronized Iterator allKeys() {
-return keySetIterator(cache.navigableKeySet());
+synchronized Iterator> allIterator() {
+return cache.entrySet().iterator();
 }
 
 synchronized LRUCacheEntry first() {
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
index 941b522..0db6c78 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
@@ -20,6 +20,7 @@ import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.common.utils.LogContext;
 import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.internals.NamedCache.LRUNode;
 import org.slf4j.Logger;
 
 import java.util.Collections;
@@ -180,17 +181,17 @@ public class ThreadCache {
 public MemoryLRUCacheBytesIterator range(final String namespace, final 
Bytes from, final Bytes to) {
 final NamedCache cache = getCache(namespace);
 if (cache == null) {
-return new 
MemoryLRUCacheBytesIterator(Collections.emptyIterator(), new 
NamedCache(namespace, this.metrics));
+return new 
MemoryLRUCacheBytesIterator(Collections.emptyIterator());
 }
-return new MemoryLRUCacheBytesIterator(cache.keyRange(from, to), 
cache);
+return new MemoryLRUCacheBytesIterator(cache.subMapIterator(from, to));
 }
 
 public MemoryLRUCacheBytesIterator all(final String namespace) {
 final NamedCache cache = getCache(namespace);
 if (cache == null) {
-return new 
MemoryLRUCacheBytesIterator(Collections.emptyIterator(), new 
NamedCache(namespace, this.metrics));
+return new 
MemoryLRUCacheBytesIterator(Collections.emptyIterator());
 }
-return new MemoryLRUCacheBytesIterator(cache.allKeys(), cache);
+return n

[kafka] branch trunk updated: MINOR: update docs JSON serde links (#6465)

2019-03-19 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 5092b26  MINOR: update docs JSON serde links (#6465)
5092b26 is described below

commit 5092b26393de731e8f826fde86b7ea97c9aaba35
Author: Victoria Bialas 
AuthorDate: Tue Mar 19 15:01:59 2019 -0700

MINOR: update docs JSON serde links (#6465)

Reviewers: Joel Mamill , Matthias J. Sax 

---
 docs/streams/developer-guide/datatypes.html | 23 ++-
 1 file changed, 10 insertions(+), 13 deletions(-)

diff --git a/docs/streams/developer-guide/datatypes.html 
b/docs/streams/developer-guide/datatypes.html
index 6c7869c..ca17c0b 100644
--- a/docs/streams/developer-guide/datatypes.html
+++ b/docs/streams/developer-guide/datatypes.html
@@ -112,7 +112,7 @@
 </dependency>
 
 
-This artifact provides the following serde implementations under 
the package https://github.com/apache/kafka/blob/1.0/clients/src/main/java/org/apache/kafka/common/serialization";>org.apache.kafka.common.serialization,
 which you can leverage when e.g., defining default serializers in your Streams 
configuration.
+This artifact provides the following serde implementations under 
the package https://github.com/apache/kafka/blob/{{dotVersion}}/clients/src/main/java/org/apache/kafka/common/serialization";>org.apache.kafka.common.serialization,
 which you can leverage when e.g., defining default serializers in your Streams 
configuration.
 
   
 
@@ -149,20 +149,17 @@
 
 
   Tip
-  https://github.com/apache/kafka/blob/1.0/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java";>Bytes
 is a wrapper for Java’s byte[] (byte array) that supports proper equality and 
ordering semantics.  You may want to consider using Bytes instead of https://github.com/apache/kafka/blob/{{dotVersion}}/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java";>Bytes
 is a wrapper for Java’s byte[] (byte array) that supports proper equality and 
ordering semantics.  You may want to consider using Bytes instead of 
 JSON
-The code examples of Kafka Streams also include a basic serde 
implementation for JSON:
+The Kafka Streams code examples also include a basic serde 
implementation for JSON:
 
-  https://github.com/apache/kafka/blob/1.0/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJOSerializer.java";>JsonPOJOSerializer
-  https://github.com/apache/kafka/blob/1.0/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonPOJODeserializer.java";>JsonPOJODeserializer
+  https://github.com/apache/kafka/blob/{{dotVersion}}/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java#L83";>PageViewTypedDemo
 
-You can construct a unified JSON serde from the JsonPOJOSerializer and 
JsonPOJODeserializer via
-  Serdes.serdeFrom(<serializerInstance>, <deserializerInstance>).  The
-  https://github.com/apache/kafka/blob/1.0/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java";>PageViewTypedDemo
-  example demonstrates how to use this JSON serde.
+As shown in the example, you can use JSONSerdes inner classes Serdes.serdeFrom(<serializerInstance>, 
<deserializerInstance>) to construct JSON compatible 
serializers and deserializers.
+
   
   
 Implementing custom 
SerDes
@@ -170,13 +167,13 @@
   existing SerDes (see previous section).  Typically, your workflow 
will be similar to:
 
   Write a serializer for your data type T by implementing
-https://github.com/apache/kafka/blob/1.0/clients/src/main/java/org/apache/kafka/common/serialization/Serializer.java";>org.apache.kafka.common.serialization.Serializer.
+https://github.com/apache/kafka/blob/{{dotVersion}}/clients/src/main/java/org/apache/kafka/common/serialization/Serializer.java";>org.apache.kafka.common.serialization.Serializer.
   Write a deserializer for T by implementing
-https://github.com/apache/kafka/blob/1.0/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java";>org.apache.kafka.common.serialization.Deserializer.
+https://github.com/apache/kafka/blob/{{dotVersion}}/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java";>org.apache.kafka.common.serialization.Deserializer.
   Write a serde for T by implementing
-https://github.com/apache/kafka/blob/1.0/clients/src/main/java/org/apache/kafka/common/serialization/Serde.java&qu

[kafka] branch trunk updated: KAFKA-6958: Add new NamedOperation interface to enforce consistency in naming operations (#6409)

2019-03-19 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new fa57eb0  KAFKA-6958: Add new NamedOperation interface to enforce 
consistency in naming operations (#6409)
fa57eb0 is described below

commit fa57eb065d032c225f63a0b2ca3f050e728c2235
Author: Florian Hussonnois 
AuthorDate: Wed Mar 20 03:27:03 2019 +0100

KAFKA-6958: Add new NamedOperation interface to enforce consistency in 
naming operations (#6409)

Sub-task required to allow to define custom processor names with KStreams 
DSL(KIP-307) :
  - add new public interface NamedOperation
  - deprecate methods Joined.as() and Joined.name()
  - update Suppredded interface to extend NamedOperation

Reviewers: Matthias J. Sax ,  John Roesler 
, Bill Bejeck 
---
 .../org/apache/kafka/streams/kstream/Joined.java   | 40 ---
 .../kafka/streams/kstream/NamedOperation.java  | 32 +++
 .../apache/kafka/streams/kstream/Suppressed.java   |  3 +-
 .../streams/kstream/internals/JoinedInternal.java  | 45 ++
 .../streams/kstream/internals/KStreamImpl.java | 15 ++--
 5 files changed, 124 insertions(+), 11 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Joined.java 
b/streams/src/main/java/org/apache/kafka/streams/kstream/Joined.java
index aa29c68..1343487 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/Joined.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Joined.java
@@ -22,13 +22,12 @@ import org.apache.kafka.common.serialization.Serde;
  * The {@code Joined} class represents optional params that can be passed to
  * {@link KStream#join}, {@link KStream#leftJoin}, and  {@link 
KStream#outerJoin} operations.
  */
-public class Joined {
-
-private final Serde keySerde;
-private final Serde valueSerde;
-private final Serde otherValueSerde;
-private final String name;
+public class Joined implements NamedOperation> {
 
+protected final Serde keySerde;
+protected final Serde valueSerde;
+protected final Serde otherValueSerde;
+protected final String name;
 
 private Joined(final Serde keySerde,
final Serde valueSerde,
@@ -40,6 +39,10 @@ public class Joined {
 this.name = name;
 }
 
+protected Joined(final Joined joined) {
+this(joined.keySerde, joined.valueSerde, joined.otherValueSerde, 
joined.name);
+}
+
 /**
  * Create an instance of {@code Joined} with key, value, and otherValue 
{@link Serde} instances.
  * {@code null} values are accepted and will be replaced by the default 
serdes as defined in config.
@@ -135,11 +138,30 @@ public class Joined {
  * @param  value type
  * @param  other value type
  * @return new {@code Joined} instance configured with the name
+ *
+ * @deprecated use {@link #as(String)} instead
  */
+@Deprecated
 public static  Joined named(final String name) {
 return new Joined<>(null, null, null, name);
 }
 
+/**
+ * Create an instance of {@code Joined} with base name for all components 
of the join, this may
+ * include any repartition topics created to complete the join.
+ *
+ * @param name the name used as the base for naming components of the join 
including any
+ * repartition topics
+ * @param  key type
+ * @param  value type
+ * @param  other value type
+ * @return new {@code Joined} instance configured with the name
+ *
+ */
+public static  Joined as(final String name) {
+return new Joined<>(null, null, null, name);
+}
+
 
 /**
  * Set the key {@link Serde} to be used. Null values are accepted and will 
be replaced by the default
@@ -182,6 +204,7 @@ public class Joined {
  * repartition topics
  * @return new {@code Joined} instance configured with the {@code name}
  */
+@Override
 public Joined withName(final String name) {
 return new Joined<>(keySerde, valueSerde, otherValueSerde, name);
 }
@@ -198,7 +221,12 @@ public class Joined {
 return otherValueSerde;
 }
 
+/**
+ * @deprecated this method will be removed in a in a future release
+ */
+@Deprecated
 public String name() {
 return name;
 }
+
 }
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/NamedOperation.java 
b/streams/src/main/java/org/apache/kafka/streams/kstream/NamedOperation.java
new file mode 100644
index 000..9a2c40b
--- /dev/null
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/NamedOperation.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for a

[kafka] branch trunk updated: MINOR: Add verification step for Streams archetype to Jenkins build (#6431)

2019-03-21 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 4cae452  MINOR: Add verification step for Streams archetype to Jenkins 
build (#6431)
4cae452 is described below

commit 4cae4523fc04597814d85c1484ec67d8cadc2256
Author: Matthias J. Sax 
AuthorDate: Thu Mar 21 12:51:18 2019 -0700

MINOR: Add verification step for Streams archetype to Jenkins build (#6431)

Updates ./jenkins.sh to build stream archetype and install it in local 
maven cache. Afterward, archetype is used to create a new maven project and 
maven project is compiled for verification.

Reviewers: Guozhang Wang , John Roesler 
,  Bill Bejeck 
---
 docs/streams/tutorial.html |  8 ++--
 jenkins.sh | 42 ++
 2 files changed, 48 insertions(+), 2 deletions(-)

diff --git a/docs/streams/tutorial.html b/docs/streams/tutorial.html
index 0006e3e..c5db280 100644
--- a/docs/streams/tutorial.html
+++ b/docs/streams/tutorial.html
@@ -74,8 +74,12 @@
 
 
 
-The pom.xml file included in the project already has the 
Streams dependency defined,
-and there are already several example programs written with Streams 
library under src/main/java.
+The pom.xml file included in the project already has the 
Streams dependency defined.
+Note, that the generated pom.xml targets Java 8, and does 
not work with higher Java versions.
+
+
+
+There are already several example programs written with Streams 
library under src/main/java.
 Since we are going to start writing such programs from scratch, we can 
now delete these examples:
 
 
diff --git a/jenkins.sh b/jenkins.sh
index 5d549fe..6f94134 100755
--- a/jenkins.sh
+++ b/jenkins.sh
@@ -28,3 +28,45 @@
 ./gradlew unitTest integrationTest \
 --profile --no-daemon --continue 
-PtestLoggingEvents=started,passed,skipped,failed "$@" \
 || { echo 'Test steps failed'; exit 1; }
+
+# Verify that Kafka Streams archetype compiles
+if [ $JAVA_HOME = "/home/jenkins/tools/java/latest11" ] ; then
+  echo "Skipping Kafka Streams archetype test for Java 11"
+  exit 0
+fi
+
+./gradlew streams:install clients:install connect:json:install 
connect:api:install \
+|| { echo 'Could not install kafka-streams.jar (and dependencies) 
locally`'; exit 1; }
+
+version=`grep "^version=" gradle.properties | cut -d= -f 2` \
+|| { echo 'Could not get version from `gradle.properties`'; exit 1; }
+
+cd streams/quickstart \
+|| { echo 'Could not change into directory `streams/quickstart`'; exit 1; }
+
+# variable $MAVEN_LATEST__HOME is provided by Jenkins (see build configuration)
+mvn=$MAVEN_LATEST__HOME/bin/mvn
+
+$mvn clean install -Dgpg.skip  \
+|| { echo 'Could not `mvn install` streams quickstart archetype'; exit 1; }
+
+mkdir test-streams-archetype && cd test-streams-archetype \
+|| { echo 'Could not create test directory for stream quickstart 
archetype'; exit 1; }
+
+echo "Y" | $mvn archetype:generate \
+-DarchetypeCatalog=local \
+-DarchetypeGroupId=org.apache.kafka \
+-DarchetypeArtifactId=streams-quickstart-java \
+-DarchetypeVersion=$version \
+-DgroupId=streams.examples \
+-DartifactId=streams.examples \
+-Dversion=0.1 \
+-Dpackage=myapps \
+|| { echo 'Could not create new project using streams quickstart 
archetype'; exit 1; }
+
+cd streams.examples \
+|| { echo 'Could not change into directory `streams.examples`'; exit 1; }
+
+$mvn compile \
+|| { echo 'Could not compile streams quickstart archetype project'; exit 
1; }
+



[kafka] branch trunk updated: MINOR: Clean up ThreadCacheTest (#6485)

2019-03-22 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 1acae2a  MINOR: Clean up ThreadCacheTest (#6485)
1acae2a is described below

commit 1acae2a67c8fce071bfe7a373187bdff209f1705
Author: Bill Bejeck 
AuthorDate: Fri Mar 22 09:27:58 2019 -0400

MINOR: Clean up ThreadCacheTest (#6485)

Minor clean up ofThreadCacheTest
Reviewers: Guozhang Wang , Matthias J. Sax 

---
 .../streams/state/internals/ThreadCacheTest.java   | 96 +-
 1 file changed, 22 insertions(+), 74 deletions(-)

diff --git 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java
index 5882ee4..c9c5789 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java
@@ -146,7 +146,7 @@ public class ThreadCacheTest {
 }
 
 @Test
-public void evict() throws IOException {
+public void evict() {
 final List> received = new ArrayList<>();
 final List> expected = 
Collections.singletonList(
 new KeyValue<>("K1", "V1"));
@@ -161,14 +161,10 @@ public class ThreadCacheTest {
 final ThreadCache cache = new ThreadCache(logContext,
   
memoryCacheEntrySize(kv.key.getBytes(), kv.value.getBytes(), ""),
   new MockStreamsMetrics(new 
Metrics()));
-cache.addDirtyEntryFlushListener(namespace, new 
ThreadCache.DirtyEntryFlushListener() {
-@Override
-public void apply(final List dirty) {
-for (final ThreadCache.DirtyEntry dirtyEntry : dirty) {
-received.add(new KeyValue<>(dirtyEntry.key().toString(), 
new String(dirtyEntry.newValue(;
-}
+cache.addDirtyEntryFlushListener(namespace, dirty -> {
+for (final ThreadCache.DirtyEntry dirtyEntry : dirty) {
+received.add(new KeyValue<>(dirtyEntry.key().toString(), new 
String(dirtyEntry.newValue(;
 }
-
 });
 
 for (final KeyValue kvToInsert : toInsert) {
@@ -200,12 +196,7 @@ public class ThreadCacheTest {
 final Bytes key = Bytes.wrap(new byte[]{0});
 final ThreadCache cache = new ThreadCache(logContext, 1L, new 
MockStreamsMetrics(new Metrics()));
 final List received = new ArrayList<>();
-cache.addDirtyEntryFlushListener(namespace, new 
ThreadCache.DirtyEntryFlushListener() {
-@Override
-public void apply(final List dirty) {
-received.addAll(dirty);
-}
-});
+cache.addDirtyEntryFlushListener(namespace, received::addAll);
 cache.put(namespace, key, dirtyEntry(key.get()));
 assertEquals(key.get(), cache.delete(namespace, key).value());
 
@@ -298,12 +289,8 @@ public class ThreadCacheTest {
 public void shouldSkipEntriesWhereValueHasBeenEvictedFromCache() {
 final int entrySize = memoryCacheEntrySize(new byte[1], new byte[1], 
"");
 final ThreadCache cache = new ThreadCache(logContext, entrySize * 5, 
new MockStreamsMetrics(new Metrics()));
-cache.addDirtyEntryFlushListener(namespace, new 
ThreadCache.DirtyEntryFlushListener() {
-@Override
-public void apply(final List dirty) {
+cache.addDirtyEntryFlushListener(namespace, dirty -> { });
 
-}
-});
 final byte[][] bytes = {{0}, {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, 
{9}};
 for (int i = 0; i < 5; i++) {
 cache.put(namespace, Bytes.wrap(bytes[i]), dirtyEntry(bytes[i]));
@@ -322,12 +309,9 @@ public class ThreadCacheTest {
 public void shouldFlushDirtyEntriesForNamespace() {
 final ThreadCache cache = new ThreadCache(logContext, 10, new 
MockStreamsMetrics(new Metrics()));
 final List received = new ArrayList<>();
-cache.addDirtyEntryFlushListener(namespace1, new 
ThreadCache.DirtyEntryFlushListener() {
-@Override
-public void apply(final List dirty) {
-for (final ThreadCache.DirtyEntry dirtyEntry : dirty) {
-received.add(dirtyEntry.key().get());
-}
+cache.addDirtyEntryFlushListener(namespace1, dirty -> {
+for (final ThreadCache.DirtyEntry dirtyEntry : dirty) {
+received.add(dirtyEntry.key().get());
 }
 });
 final List expected = Arrays.asList(new byte[]{0}, new 
byte[]{1}, new byte[]{2});
@@ -344,12 +328,9 @@ public clas

[kafka] branch trunk updated: MINOR: Remove line for testing repartition topic name (#6488)

2019-03-24 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new c74acb2  MINOR: Remove line for testing repartition topic name (#6488)
c74acb2 is described below

commit c74acb24eb1da5d16cc2721a63931cd87f79ec66
Author: Bill Bejeck 
AuthorDate: Sun Mar 24 12:47:58 2019 -0400

MINOR: Remove line for testing repartition topic name (#6488)

With KIP-307 joined.name() is deprecated plus we don't need to test for 
repartition topic names.
Reviewers: Matthias J. Sax 
---
 .../test/scala/org/apache/kafka/streams/scala/kstream/JoinedTest.scala   | 1 -
 1 file changed, 1 deletion(-)

diff --git 
a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/JoinedTest.scala
 
b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/JoinedTest.scala
index 9a96a81..f523e20 100644
--- 
a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/JoinedTest.scala
+++ 
b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/JoinedTest.scala
@@ -42,6 +42,5 @@ class JoinedTest extends FlatSpec with Matchers {
 joined.keySerde.getClass shouldBe Serdes.String.getClass
 joined.valueSerde.getClass shouldBe Serdes.Long.getClass
 joined.otherValueSerde.getClass shouldBe Serdes.Integer.getClass
-joined.name() shouldBe repartitionTopicName
   }
 }



[kafka] branch 1.1 updated: KAFKA-8026: Fix for flaky RegexSourceIntegrationTest (#6459)

2019-03-25 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 1.1
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/1.1 by this push:
 new 92c591d  KAFKA-8026: Fix for flaky RegexSourceIntegrationTest (#6459)
92c591d is described below

commit 92c591d6d49d26b489eb71875b62d533b4f1e5ab
Author: Bill Bejeck 
AuthorDate: Mon Mar 25 12:23:57 2019 -0400

KAFKA-8026: Fix for flaky RegexSourceIntegrationTest (#6459)

Reviewers:  Guozhang Wang , Matthias J. Sax 

---
 .../integration/RegexSourceIntegrationTest.java| 201 -
 1 file changed, 112 insertions(+), 89 deletions(-)

diff --git 
a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
index 5f0a107..4b776e9 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
@@ -44,7 +44,6 @@ import org.apache.kafka.test.TestCondition;
 import org.apache.kafka.test.TestUtils;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -57,7 +56,8 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
-import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 
 import static org.hamcrest.CoreMatchers.equalTo;
@@ -90,12 +90,13 @@ public class RegexSourceIntegrationTest {
 private static final String STRING_SERDE_CLASSNAME = 
Serdes.String().getClass().getName();
 private Properties streamsConfiguration;
 private static final String STREAM_TASKS_NOT_UPDATED = "Stream tasks not 
updated";
-private KafkaStreams streams;
 
+@Before
+public void setUp() throws InterruptedException {
+final Properties properties = new Properties();
+properties.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, 
true);
 
-@BeforeClass
-public static void startKafkaCluster() throws InterruptedException {
-CLUSTER.createTopics(
+CLUSTER.deleteAndRecreateTopics(
 TOPIC_1,
 TOPIC_2,
 TOPIC_A,
@@ -105,26 +106,22 @@ public class RegexSourceIntegrationTest {
 FA_TOPIC,
 FOO_TOPIC,
 DEFAULT_OUTPUT_TOPIC);
+
+CLUSTER.deleteTopicsAndWait(PARTITIONED_TOPIC_1, PARTITIONED_TOPIC_2);
+
 CLUSTER.createTopic(PARTITIONED_TOPIC_1, 2, 1);
 CLUSTER.createTopic(PARTITIONED_TOPIC_2, 2, 1);
-}
 
-@Before
-public void setUp() {
-final Properties properties = new Properties();
-properties.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, 
true);
-streamsConfiguration = 
StreamsTestUtils.getStreamsConfig("regex-source-integration-test",
- 
CLUSTER.bootstrapServers(),
- 
STRING_SERDE_CLASSNAME,
- 
STRING_SERDE_CLASSNAME,
- properties);
+streamsConfiguration = StreamsTestUtils.getStreamsConfig(
+UUID.randomUUID().toString(),
+CLUSTER.bootstrapServers(),
+STRING_SERDE_CLASSNAME,
+STRING_SERDE_CLASSNAME,
+properties);
 }
 
 @After
 public void tearDown() throws IOException {
-if (streams != null) {
-streams.close();
-}
 // Remove any state from previous test runs
 IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration);
 }
@@ -145,8 +142,8 @@ public class RegexSourceIntegrationTest {
 final KStream pattern1Stream = 
builder.stream(Pattern.compile("TEST-TOPIC-\\d"));
 
 pattern1Stream.to(stringSerde, stringSerde, DEFAULT_OUTPUT_TOPIC);
-final List assignedTopics = new CopyOnWriteArrayList<>();
-streams = new KafkaStreams(builder.build(), streamsConfig, new 
DefaultKafkaClientSupplier() {
+final List assignedTopics = new ArrayList<>();
+final KafkaStreams streams = new KafkaStreams(builder.build(), 
streamsConfig, new DefaultKafkaClientSupplier() {
 @Override
 public Consumer getConsumer(final Map config) {
 return new KafkaConsumer(config, new 
ByteArrayDeserializer(), new ByteArrayDeserializer()) {
@@ -159,23 +156,30 @@ public class RegexSourceIntegrationTest {
 }
 });
 

[kafka] branch 1.0 updated: KAFKA-8026: Fix flaky regex source integration test 1.0 (#6463)

2019-03-25 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 1.0
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/1.0 by this push:
 new 7434ef1  KAFKA-8026: Fix flaky regex source integration test 1.0 
(#6463)
7434ef1 is described below

commit 7434ef11d090bdf8583363e453552df6a1254f5c
Author: Bill Bejeck 
AuthorDate: Mon Mar 25 17:10:23 2019 -0400

KAFKA-8026: Fix flaky regex source integration test 1.0 (#6463)

Reviewers: Matthias J. Sax ,  John Roesler 

---
 .../integration/RegexSourceIntegrationTest.java| 200 -
 1 file changed, 111 insertions(+), 89 deletions(-)

diff --git 
a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
index 5f0a107..6773b2a 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
@@ -44,7 +44,6 @@ import org.apache.kafka.test.TestCondition;
 import org.apache.kafka.test.TestUtils;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -57,7 +56,8 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
-import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 
 import static org.hamcrest.CoreMatchers.equalTo;
@@ -90,12 +90,13 @@ public class RegexSourceIntegrationTest {
 private static final String STRING_SERDE_CLASSNAME = 
Serdes.String().getClass().getName();
 private Properties streamsConfiguration;
 private static final String STREAM_TASKS_NOT_UPDATED = "Stream tasks not 
updated";
-private KafkaStreams streams;
 
+@Before
+public void setUp() throws InterruptedException {
+final Properties properties = new Properties();
+properties.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, 
true);
 
-@BeforeClass
-public static void startKafkaCluster() throws InterruptedException {
-CLUSTER.createTopics(
+CLUSTER.deleteAndRecreateTopics(
 TOPIC_1,
 TOPIC_2,
 TOPIC_A,
@@ -105,26 +106,21 @@ public class RegexSourceIntegrationTest {
 FA_TOPIC,
 FOO_TOPIC,
 DEFAULT_OUTPUT_TOPIC);
+
+CLUSTER.deleteTopicsAndWait(PARTITIONED_TOPIC_1, PARTITIONED_TOPIC_2);
+
 CLUSTER.createTopic(PARTITIONED_TOPIC_1, 2, 1);
 CLUSTER.createTopic(PARTITIONED_TOPIC_2, 2, 1);
-}
 
-@Before
-public void setUp() {
-final Properties properties = new Properties();
-properties.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, 
true);
-streamsConfiguration = 
StreamsTestUtils.getStreamsConfig("regex-source-integration-test",
- 
CLUSTER.bootstrapServers(),
- 
STRING_SERDE_CLASSNAME,
- 
STRING_SERDE_CLASSNAME,
- properties);
+streamsConfiguration = 
StreamsTestUtils.getStreamsConfig(UUID.randomUUID().toString(),
+CLUSTER.bootstrapServers(),
+STRING_SERDE_CLASSNAME,
+STRING_SERDE_CLASSNAME,
+properties);
 }
 
 @After
 public void tearDown() throws IOException {
-if (streams != null) {
-streams.close();
-}
 // Remove any state from previous test runs
 IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration);
 }
@@ -145,8 +141,8 @@ public class RegexSourceIntegrationTest {
 final KStream pattern1Stream = 
builder.stream(Pattern.compile("TEST-TOPIC-\\d"));
 
 pattern1Stream.to(stringSerde, stringSerde, DEFAULT_OUTPUT_TOPIC);
-final List assignedTopics = new CopyOnWriteArrayList<>();
-streams = new KafkaStreams(builder.build(), streamsConfig, new 
DefaultKafkaClientSupplier() {
+final List assignedTopics = new ArrayList<>();
+final KafkaStreams streams = new KafkaStreams(builder.build(), 
streamsConfig, new DefaultKafkaClientSupplier() {
 @Override
 public Consumer getConsumer(final Map config) {
 return new KafkaConsumer(config, new 
ByteArrayDeserializer(), new ByteArrayDeserializer()) {
@@ -159,23 +155,30 @@ public class RegexSourceIntegrationTest {
 }
 });
 
+try {
+streams.start(

[kafka] branch trunk updated: Cleanup KTableImpl#doTransformValues (#6519)

2019-03-29 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new d10023e  Cleanup KTableImpl#doTransformValues (#6519)
d10023e is described below

commit d10023e8d3093eaa6b0b3afa04c7ea1b9bee6c30
Author: Lee Dongjin 
AuthorDate: Sat Mar 30 06:08:20 2019 +0900

Cleanup KTableImpl#doTransformValues (#6519)

This PR is a follow-up of #6174 and #6453, which cleans up 
KTableImpl#doTransformValues method.

Reviewers: Bill Bejeck 
---
 .../streams/kstream/internals/KTableImpl.java  | 31 +++---
 1 file changed, 22 insertions(+), 9 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
index 67b3c31..e9291cc 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
@@ -295,12 +295,28 @@ public class KTableImpl extends 
AbstractStream implements KTable<
  final MaterializedInternal> materializedInternal,
  final String... 
stateStoreNames) {
 Objects.requireNonNull(stateStoreNames, "stateStoreNames");
+final Serde keySerde;
+final Serde valueSerde;
+final String queryableStoreName;
+final StoreBuilder> storeBuilder;
 
-final String name = builder.newProcessorName(TRANSFORMVALUES_NAME);
+if (materializedInternal != null) {
+// don't inherit parent value serde, since this operation may 
change the value type, more specifically:
+// we preserve the key following the order of 1) materialized, 2) 
parent, 3) null
+keySerde = materializedInternal.keySerde() != null ? 
materializedInternal.keySerde() : this.keySerde;
+// we preserve the value following the order of 1) materialized, 
2) null
+valueSerde = materializedInternal.valueSerde();
+queryableStoreName = materializedInternal.queryableStoreName();
+// only materialize if materialized is specified and it has 
queryable name
+storeBuilder = queryableStoreName != null ? (new 
KeyValueStoreMaterializer<>(materializedInternal)).materialize() : null;
+} else {
+keySerde = this.keySerde;
+valueSerde = null;
+queryableStoreName = null;
+storeBuilder = null;
+}
 
-// only materialize if users provide a specific queryable name
-final String queryableStoreName = materializedInternal != null ? 
materializedInternal.queryableStoreName() : null;
-final StoreBuilder> storeBuilder = 
queryableStoreName != null ? (new 
KeyValueStoreMaterializer<>(materializedInternal)).materialize() : null;
+final String name = builder.newProcessorName(TRANSFORMVALUES_NAME);
 
 final KTableProcessorSupplier processorSupplier = new 
KTableTransformValues<>(
 this,
@@ -320,13 +336,10 @@ public class KTableImpl extends 
AbstractStream implements KTable<
 
 builder.addGraphNode(this.streamsGraphNode, tableNode);
 
-// don't inherit parent value serde, since this operation may change 
the value type, more specifically:
-// we preserve the key following the order of 1) materialized, 2) 
parent, 3) null
-// we preserve the value following the order of 1) materialized, 2) 
null
 return new KTableImpl<>(
 name,
-materializedInternal != null && materializedInternal.keySerde() != 
null ? materializedInternal.keySerde() : keySerde,
-materializedInternal != null ? materializedInternal.valueSerde() : 
null,
+keySerde,
+valueSerde,
 sourceNodes,
 queryableStoreName,
 processorSupplier,



[kafka] branch trunk updated: KAFKA-7502: Cleanup KTable materialization logic in a single place (doMapValues) (#6520)

2019-03-29 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new ff78c68  KAFKA-7502: Cleanup KTable materialization logic in a single 
place (doMapValues) (#6520)
ff78c68 is described below

commit ff78c684ff22d81174cd789a4ac7e7e1fe4dfc8a
Author: Lee Dongjin 
AuthorDate: Sat Mar 30 06:10:04 2019 +0900

KAFKA-7502: Cleanup KTable materialization logic in a single place 
(doMapValues) (#6520)

* Move materialization logic from TableProcessorNode to KTableImpl

1. TableProcessorNode: remove materializedInternal, use storeBuilder 
instead.
2. Instantiate StoreBuilder in KTableImpl#[doFilter, doMapValues, 
doTransformValues], instead of TableProcessorNode#init.

* Cleanup KTableImpl#doMapValues

* 1. Add TableProcessorNode(String, ProcessorParameters, StoreBuilder). 2. 
Reformat+trivial changes on TableProcessorNode.java.
---
 .../streams/kstream/internals/KTableImpl.java  | 32 --
 1 file changed, 23 insertions(+), 9 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
index e9291cc..6a1af18 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
@@ -201,17 +201,31 @@ public class KTableImpl extends 
AbstractStream implements KTable<
 
 private  KTable doMapValues(final ValueMapperWithKey mapper,
final MaterializedInternal> materializedInternal) {
-// we actually do not need generate store names at all since if it is 
not specified, we will not
-// materialize the store; but we still need to burn one index BEFORE 
generating the processor to keep compatibility.
-if (materializedInternal != null && materializedInternal.storeName() 
== null) {
-builder.newStoreName(MAPVALUES_NAME);
+final Serde keySerde;
+final Serde valueSerde;
+final String queryableStoreName;
+final StoreBuilder> storeBuilder;
+
+if (materializedInternal != null) {
+// we actually do not need to generate store names at all since if 
it is not specified, we will not
+// materialize the store; but we still need to burn one index 
BEFORE generating the processor to keep compatibility.
+if (materializedInternal.storeName() == null) {
+builder.newStoreName(MAPVALUES_NAME);
+}
+keySerde = materializedInternal.keySerde() != null ? 
materializedInternal.keySerde() : this.keySerde;
+valueSerde = materializedInternal.valueSerde();
+queryableStoreName = materializedInternal.queryableStoreName();
+// only materialize if materialized is specified and it has 
queryable name
+storeBuilder = queryableStoreName != null ? (new 
KeyValueStoreMaterializer<>(materializedInternal)).materialize() : null;
+} else {
+keySerde = this.keySerde;
+valueSerde = null;
+queryableStoreName = null;
+storeBuilder = null;
 }
 
 final String name = builder.newProcessorName(MAPVALUES_NAME);
 
-// only materialize if the state store has queryable name
-final String queryableStoreName = materializedInternal != null ? 
materializedInternal.queryableStoreName() : null;
-final StoreBuilder> storeBuilder = 
queryableStoreName != null ? (new 
KeyValueStoreMaterializer<>(materializedInternal)).materialize() : null;
 final KTableProcessorSupplier processorSupplier = new 
KTableMapValues<>(this, mapper, queryableStoreName);
 
 // leaving in calls to ITB until building topology with graph
@@ -232,8 +246,8 @@ public class KTableImpl extends AbstractStream implements KTable<
 // we preserve the value following the order of 1) materialized, 2) 
null
 return new KTableImpl<>(
 name,
-materializedInternal != null && materializedInternal.keySerde() != 
null ? materializedInternal.keySerde() : keySerde,
-materializedInternal != null ? materializedInternal.valueSerde() : 
null,
+keySerde,
+valueSerde,
 sourceNodes,
 queryableStoreName,
 processorSupplier,



[kafka] branch trunk updated: MINOR: Move KTable source topic for changelog to optimization framework (#6500)

2019-03-29 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 369d89f  MINOR: Move KTable source topic for changelog to optimization 
framework (#6500)
369d89f is described below

commit 369d89f2080586773b5a3cc432cbcf655aa1f625
Author: Bill Bejeck 
AuthorDate: Fri Mar 29 17:16:56 2019 -0400

MINOR: Move KTable source topic for changelog to optimization framework 
(#6500)

Since we've added Kafka Streams optimizations in 2.1 we need to move the 
optimization for source KTable nodes (use source topic as changelog) to the 
optimization framework.

Reviewers: Guozhang Wang 
---
 .../kstream/internals/InternalStreamsBuilder.java  |  9 ++
 .../kstream/internals/graph/TableSourceNode.java   | 12 +++-
 .../internals/InternalTopologyBuilder.java | 36 +-
 .../apache/kafka/streams/StreamsBuilderTest.java   |  2 +-
 .../integration/RestoreIntegrationTest.java|  2 +-
 5 files changed, 23 insertions(+), 38 deletions(-)

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 c06b988..920f213 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
@@ -61,6 +61,7 @@ public class InternalStreamsBuilder implements 
InternalNameProvider {
 private final AtomicInteger buildPriorityIndex = new AtomicInteger(0);
 private final LinkedHashMap> 
keyChangingOperationsToOptimizableRepartitionNodes = new LinkedHashMap<>();
 private final LinkedHashSet mergeNodes = new 
LinkedHashSet<>();
+private final LinkedHashSet tableSourceNodes = new 
LinkedHashSet<>();
 
 private static final String TOPOLOGY_ROOT = "root";
 private static final Logger LOG = 
LoggerFactory.getLogger(InternalStreamsBuilder.class);
@@ -254,6 +255,8 @@ public class InternalStreamsBuilder implements 
InternalNameProvider {
 }
 } else if (node.isMergeNode()) {
 mergeNodes.add(node);
+} else if (node instanceof TableSourceNode) {
+tableSourceNodes.add(node);
 }
 }
 
@@ -292,10 +295,16 @@ public class InternalStreamsBuilder implements 
InternalNameProvider {
 
 if (props != null && 
StreamsConfig.OPTIMIZE.equals(props.getProperty(StreamsConfig.TOPOLOGY_OPTIMIZATION)))
 {
 LOG.debug("Optimizing the Kafka Streams graph for repartition 
nodes");
+optimizeKTableSourceTopics();
 maybeOptimizeRepartitionOperations();
 }
 }
 
+private void optimizeKTableSourceTopics() {
+LOG.debug("Marking KTable source nodes to optimize using source topic 
for changelogs ");
+tableSourceNodes.forEach(node -> ((TableSourceNode) 
node).reuseSourceTopicForChangeLog(true));
+}
+
 @SuppressWarnings("unchecked")
 private void maybeOptimizeRepartitionOperations() {
 maybeUpdateKeyChangingRepartitionNodeMap();
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java
index 53061dc..fa979b2 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java
@@ -38,6 +38,7 @@ public class TableSourceNode extends 
StreamSourceNode {
 private final ProcessorParameters processorParameters;
 private final String sourceName;
 private final boolean isGlobalKTable;
+private boolean shouldReuseSourceTopicForChangelog = false;
 
 private TableSourceNode(final String nodeName,
 final String sourceName,
@@ -57,6 +58,11 @@ public class TableSourceNode extends 
StreamSourceNode {
 this.materializedInternal = materializedInternal;
 }
 
+
+public void reuseSourceTopicForChangeLog(final boolean 
shouldReuseSourceTopicForChangelog) {
+this.shouldReuseSourceTopicForChangelog = 
shouldReuseSourceTopicForChangelog;
+}
+
 @Override
 public String toString() {
 return "TableSourceNode{" +
@@ -104,7 +110,11 @@ public class TableSourceNode extends 
StreamSourceNode {
 final KTableSource ktableSource = (KTableSource) 
processorParameters.processorSupplier();
 if (ktableSource.queryableName() != null) {
 topologyBuilder.addStateStore(storeBuilder, nodeName());
-t

[kafka] branch trunk updated: fix compile error for example (#6526)

2019-04-01 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new acb5188  fix compile error for example (#6526)
acb5188 is described below

commit acb5188810320224713babbd03eb3709f6e7968e
Author: Jarrod Urban 
AuthorDate: Mon Apr 1 17:39:05 2019 +0200

fix compile error for example (#6526)

Reviewers: Prashant Sabnekar, Bill Bejeck 
---
 docs/streams/index.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/streams/index.html b/docs/streams/index.html
index 193a7b2..6872297 100644
--- a/docs/streams/index.html
+++ b/docs/streams/index.html
@@ -275,7 +275,7 @@ object WordCountApplication extends App {
   val wordCounts: KTable[String, Long] = textLines
 .flatMapValues(textLine => textLine.toLowerCase.split("\\W+"))
 .groupBy((_, word) => word)
-.count(Materialized.as("counts-store"))
+.count()(Materialized.as("counts-store"))
   wordCounts.toStream.to("WordsWithCountsTopic")
 
   val streams: KafkaStreams = new KafkaStreams(builder.build(), props)



[kafka] branch trunk updated: KAFKA-6399: Remove Streams max.poll.interval override (#6509)

2019-04-04 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 9bd0d6a  KAFKA-6399: Remove Streams max.poll.interval override (#6509)
9bd0d6a is described below

commit 9bd0d6aa93b901be97adb53f290b262c7cf1f175
Author: John Roesler 
AuthorDate: Thu Apr 4 08:38:25 2019 -0500

KAFKA-6399: Remove Streams max.poll.interval override (#6509)

Since we now call poll during restore, we can decrease the timeout
to a reasonable value, which should help Streams make progress if
threads get stuck.

Reviewers: Guozhang Wang ,  Bill Bejeck 

---
 streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java | 7 ---
 1 file changed, 7 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java 
b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
index f607d1d..f3e6386 100644
--- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
+++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
@@ -719,13 +719,6 @@ public class StreamsConfig extends AbstractConfig {
 
tempConsumerDefaultOverrides.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, 
"earliest");
 
tempConsumerDefaultOverrides.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, 
"false");
 tempConsumerDefaultOverrides.put("internal.leave.group.on.close", 
false);
-// MAX_POLL_INTERVAL_MS_CONFIG needs to be large for streams to handle 
cases when
-// streams is recovering data from state stores. We may set it to 
Integer.MAX_VALUE since
-// the streams code itself catches most exceptions and acts 
accordingly without needing
-// this timeout. Note however that deadlocks are not detected (by 
definition) so we
-// are losing the ability to detect them by setting this value to 
large. Hopefully
-// deadlocks happen very rarely or never.
-
tempConsumerDefaultOverrides.put(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 
Integer.toString(Integer.MAX_VALUE));
 CONSUMER_DEFAULT_OVERRIDES = 
Collections.unmodifiableMap(tempConsumerDefaultOverrides);
 }
 



[kafka] branch trunk updated: [MINOR] Guard against crashing on invalid key range queries (#6521)

2019-04-10 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 9f5a69a  [MINOR] Guard against crashing on invalid key range queries 
(#6521)
9f5a69a is described below

commit 9f5a69a4c2d6ac812ab6134e64839602a0840b87
Author: A. Sophie Blee-Goldman 
AuthorDate: Wed Apr 10 12:10:47 2019 -0700

[MINOR] Guard against crashing on invalid key range queries (#6521)

Due to KAFKA-8159, Streams will throw an unchecked exception when a caching 
layer or in-memory underlying store is queried over a range of keys from 
negative to positive. We should add a check for this and log it then return an 
empty iterator (as the RocksDB stores happen to do) rather than crash

Reviewers: Bruno Cadonna  Bill Bejeck 

---
 .../AbstractRocksDBSegmentedBytesStore.java|  7 
 .../state/internals/CachingKeyValueStore.java  | 11 ++
 .../state/internals/CachingSessionStore.java   | 11 ++
 .../state/internals/CachingWindowStore.java| 11 ++
 .../state/internals/InMemoryKeyValueStore.java | 12 +++
 .../state/internals/InMemoryWindowStore.java   |  9 -
 .../state/internals/MemoryNavigableLRUCache.java   | 12 +++
 .../streams/state/internals/RocksDBStore.java  |  8 +
 .../state/internals/AbstractKeyValueStoreTest.java | 31 +
 .../state/internals/CachingSessionStoreTest.java   | 36 
 .../state/internals/CachingWindowStoreTest.java| 39 ++
 .../state/internals/InMemoryWindowStoreTest.java   | 34 +++
 .../state/internals/RocksDBSessionStoreTest.java   | 35 ++-
 .../state/internals/RocksDBWindowStoreTest.java| 37 
 14 files changed, 291 insertions(+), 2 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java
index 34639e3..22f3a02 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java
@@ -86,6 +86,13 @@ public class AbstractRocksDBSegmentedBytesStore implements Se
  final Bytes keyTo,
  final long from,
  final long to) {
+if (keyFrom.compareTo(keyTo) > 0) {
+LOG.warn("Returning empty iterator for fetch with invalid key 
range: from > to. "
++ "This may be due to serdes that don't preserve ordering when 
lexicographically comparing the serialized bytes. " +
+"Note that the built-in numerical serdes do not follow this 
for negative numbers");
+return KeyValueIterators.emptyIterator();
+}
+
 final List searchSpace = keySchema.segmentsToSearch(segments, from, 
to);
 
 final Bytes binaryFrom = keySchema.lowerRange(keyFrom, from);
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
index bb347de..95e20b4 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
@@ -30,11 +30,15 @@ import java.util.Objects;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 class CachingKeyValueStore
 extends WrappedStateStore, byte[], byte[]>
 implements KeyValueStore, CachedStateStore {
 
+private static final Logger LOG = 
LoggerFactory.getLogger(CachingKeyValueStore.class);
+
 private CacheFlushListener flushListener;
 private boolean sendOldValues;
 private String cacheName;
@@ -228,6 +232,13 @@ class CachingKeyValueStore
 @Override
 public KeyValueIterator range(final Bytes from,
  final Bytes to) {
+if (from.compareTo(to) > 0) {
+LOG.warn("Returning empty iterator for fetch with invalid key 
range: from > to. "
++ "This may be due to serdes that don't preserve ordering when 
lexicographically comparing the serialized bytes. " +
+"Note that the built-in numerical serdes do not follow this 
for negative numbers");
+return KeyValu

[kafka] branch trunk updated: KAFKA-8209: Wrong link for KStreams DSL in core concepts doc (#6564)

2019-04-13 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 75dc7e2  KAFKA-8209: Wrong link for KStreams DSL in core concepts doc 
(#6564)
75dc7e2 is described below

commit 75dc7e2b6b5522911b22f52bdba5357b19a9a773
Author: Bill Bejeck 
AuthorDate: Sat Apr 13 18:50:27 2019 -0400

KAFKA-8209: Wrong link for KStreams DSL in core concepts doc (#6564)

Reviewers Matthias J. Sax , Michael Drogalis 
, Victoria Bialas 
---
 docs/streams/core-concepts.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html
index c925c2e..79f5b82 100644
--- a/docs/streams/core-concepts.html
+++ b/docs/streams/core-concepts.html
@@ -185,7 +185,7 @@
 Some stream processing applications don't require state, which means 
the processing of a message is independent from
 the processing of all other messages.
 However, being able to maintain state opens up many possibilities for 
sophisticated stream processing applications: you
-can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
+can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
 
 
 Kafka Streams provides so-called state stores, which can be 
used by stream processing applications to store and query data.



[kafka-site] branch asf-site updated: Fix broken link for streams DSL page (#198)

2019-04-13 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/kafka-site.git


The following commit(s) were added to refs/heads/asf-site by this push:
 new 7da40e9  Fix broken link for streams DSL page (#198)
7da40e9 is described below

commit 7da40e9b5f98cfbc6b9fb2a4c889daa4935c5f2b
Author: Bill Bejeck 
AuthorDate: Sat Apr 13 18:51:59 2019 -0400

Fix broken link for streams DSL page (#198)

Reviewers: Matthias J. Sax , Michael Drogalis 
, Victoria Bialas 
---
 10/streams/core-concepts.html | 2 +-
 11/streams/core-concepts.html | 2 +-
 20/streams/core-concepts.html | 2 +-
 21/streams/core-concepts.html | 2 +-
 22/streams/core-concepts.html | 2 +-
 5 files changed, 5 insertions(+), 5 deletions(-)

diff --git a/10/streams/core-concepts.html b/10/streams/core-concepts.html
index f2f32ad..81bfdf6 100644
--- a/10/streams/core-concepts.html
+++ b/10/streams/core-concepts.html
@@ -78,7 +78,7 @@
 
 
 
-Kafka Streams offers two ways to define the stream processing 
topology: the Kafka 
Streams DSL provides
+Kafka Streams offers two ways to define the stream processing 
topology: the Kafka 
Streams DSL provides
 the most common data transformation operations such as 
map, filter, join and 
aggregations out of the box; the lower-level Processor
 API allows
 developers define and connect custom processors as well as to interact 
with state stores.
 
diff --git a/11/streams/core-concepts.html b/11/streams/core-concepts.html
index b5c676c..473a268 100644
--- a/11/streams/core-concepts.html
+++ b/11/streams/core-concepts.html
@@ -133,7 +133,7 @@
 Some stream processing applications don't require state, which means 
the processing of a message is independent from
 the processing of all other messages.
 However, being able to maintain state opens up many possibilities for 
sophisticated stream processing applications: you
-can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
+can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
 
 
 Kafka Streams provides so-called state stores, which can be 
used by stream processing applications to store and query data.
diff --git a/20/streams/core-concepts.html b/20/streams/core-concepts.html
index b6d7762..594efaa 100644
--- a/20/streams/core-concepts.html
+++ b/20/streams/core-concepts.html
@@ -185,7 +185,7 @@
 Some stream processing applications don't require state, which means 
the processing of a message is independent from
 the processing of all other messages.
 However, being able to maintain state opens up many possibilities for 
sophisticated stream processing applications: you
-can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
+can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
 
 
 Kafka Streams provides so-called state stores, which can be 
used by stream processing applications to store and query data.
diff --git a/21/streams/core-concepts.html b/21/streams/core-concepts.html
index bffaa8a..f630772 100644
--- a/21/streams/core-concepts.html
+++ b/21/streams/core-concepts.html
@@ -185,7 +185,7 @@
 Some stream processing applications don't require state, which means 
the processing of a message is independent from
 the processing of all other messages.
 However, being able to maintain state opens up many possibilities for 
sophisticated stream processing applications: you
-can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
+can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
 
 
 Kafka Streams provides so-called state stores, which can be 
used by stream processing applications to store and query data.
diff --git a/22/streams/core-concepts.html b/22/streams/core-concepts.html
index c925c2e..79f5b82 100644
--- a/22/streams/core-concepts.html
+++ b/22/streams/core-concepts.html
@@ -185,7 +185,7 @@
 Some stream processing applications don't require state, which means 
the processing of a message is independent from
 the processing of all other messages.
 However, being able to maintain state opens up many possibilities for 
sophisticated stream processing applications: you
-can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
+can join input streams, or

[kafka] branch trunk updated: KAFKA-8210: Fix link for streams table duality (#6573)

2019-04-13 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 5e2d062  KAFKA-8210: Fix link for streams table duality (#6573)
5e2d062 is described below

commit 5e2d062267731a90a4f4c60fd2594f1f67e4752b
Author: Bill Bejeck 
AuthorDate: Sat Apr 13 18:54:06 2019 -0400

KAFKA-8210: Fix link for streams table duality (#6573)

Reviewers: Victoria Bialas 
---
 docs/streams/developer-guide/dsl-api.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/streams/developer-guide/dsl-api.html 
b/docs/streams/developer-guide/dsl-api.html
index 0ddcbc5..f5c3df9 100644
--- a/docs/streams/developer-guide/dsl-api.html
+++ b/docs/streams/developer-guide/dsl-api.html
@@ -163,7 +163,7 @@
 
 
 
-We have already seen an example of a changelog stream in the 
section streams_concepts_duality. Another example are change 
data capture (CDC) records in the changelog of a relational database, 
representing which row in a database table was inserted, updated, or deleted.
+We have already seen an example of a changelog stream in the 
section streams and tables. Another example are change data capture 
(CDC) records in the changelog of a relational database, representing which row 
in a database table was inserted, updated, or deleted.
 
 
 



[kafka-site] branch asf-site updated: Fix streams table duality links (#199)

2019-04-13 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/kafka-site.git


The following commit(s) were added to refs/heads/asf-site by this push:
 new 4f25f99  Fix streams table duality links (#199)
4f25f99 is described below

commit 4f25f994ded0c0e58b3c4d321d6f30beb650a88e
Author: Bill Bejeck 
AuthorDate: Sat Apr 13 18:55:17 2019 -0400

Fix streams table duality links (#199)

Reviewers: Victoria Bialas 
---
 20/streams/developer-guide/dsl-api.html | 2 +-
 21/streams/developer-guide/dsl-api.html | 2 +-
 22/streams/developer-guide/dsl-api.html | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/20/streams/developer-guide/dsl-api.html 
b/20/streams/developer-guide/dsl-api.html
index a348c0a..2e8af26 100644
--- a/20/streams/developer-guide/dsl-api.html
+++ b/20/streams/developer-guide/dsl-api.html
@@ -161,7 +161,7 @@
 
 
 
-We have already seen an example of a changelog stream in the 
section streams_concepts_duality. Another example are change 
data capture (CDC) records in the changelog of a relational database, 
representing which row in a database table was inserted, updated, or deleted.
+We have already seen an example of a changelog stream in the 
section streams and tables. Another example are change data capture 
(CDC) records in the changelog of a relational database, representing which row 
in a database table was inserted, updated, or deleted.
 
 
 
diff --git a/21/streams/developer-guide/dsl-api.html 
b/21/streams/developer-guide/dsl-api.html
index 423fc4f..fc45ce5 100644
--- a/21/streams/developer-guide/dsl-api.html
+++ b/21/streams/developer-guide/dsl-api.html
@@ -163,7 +163,7 @@
 
 
 
-We have already seen an example of a changelog stream in the 
section streams_concepts_duality. Another example are change 
data capture (CDC) records in the changelog of a relational database, 
representing which row in a database table was inserted, updated, or deleted.
+We have already seen an example of a changelog stream in the 
section streams and tables. Another example are change data capture 
(CDC) records in the changelog of a relational database, representing which row 
in a database table was inserted, updated, or deleted.
 
 
 
diff --git a/22/streams/developer-guide/dsl-api.html 
b/22/streams/developer-guide/dsl-api.html
index 0ddcbc5..f5c3df9 100644
--- a/22/streams/developer-guide/dsl-api.html
+++ b/22/streams/developer-guide/dsl-api.html
@@ -163,7 +163,7 @@
 
 
 
-We have already seen an example of a changelog stream in the 
section streams_concepts_duality. Another example are change 
data capture (CDC) records in the changelog of a relational database, 
representing which row in a database table was inserted, updated, or deleted.
+We have already seen an example of a changelog stream in the 
section streams and tables. Another example are change data capture 
(CDC) records in the changelog of a relational database, representing which row 
in a database table was inserted, updated, or deleted.
 
 
 



[kafka-site] branch asf-site updated: KAFKA-8208: Fix broken link for out of order data (#197)

2019-04-13 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/kafka-site.git


The following commit(s) were added to refs/heads/asf-site by this push:
 new 044b919  KAFKA-8208: Fix broken link for out of order data (#197)
044b919 is described below

commit 044b919f13357213c5ff0eec9c527ac95138df70
Author: Bill Bejeck 
AuthorDate: Sat Apr 13 18:56:35 2019 -0400

KAFKA-8208: Fix broken link for out of order data (#197)

Matthias J. Sax , Michael Drogalis 
, Victoria Bialas 
---
 21/streams/core-concepts.html | 2 +-
 22/streams/core-concepts.html | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/21/streams/core-concepts.html b/21/streams/core-concepts.html
index f630772..1e1aeb7 100644
--- a/21/streams/core-concepts.html
+++ b/21/streams/core-concepts.html
@@ -224,7 +224,7 @@
 
 
 Besides the guarantee that each record will be processed exactly-once, 
another issue that many stream processing application will face is how to
-handle out-of-order data that may impact their 
business logic. In Kafka Streams, there are two causes that could potentially
+handle https://dl.acm.org/citation.cfm?id=3242155";>out-of-order data that 
may impact their business logic. In Kafka Streams, there are two causes that 
could potentially
 result in out-of-order data arrivals with respect to their timestamps:
 
 
diff --git a/22/streams/core-concepts.html b/22/streams/core-concepts.html
index 79f5b82..1e1aeb7 100644
--- a/22/streams/core-concepts.html
+++ b/22/streams/core-concepts.html
@@ -224,7 +224,7 @@
 
 
 Besides the guarantee that each record will be processed exactly-once, 
another issue that many stream processing application will face is how to
-handle https://www.confluent.io/wp-content/uploads/streams-tables-two-sides-same-coin.pdf";>out-of-order
 data that may impact their business logic. In Kafka Streams, there are two 
causes that could potentially
+handle https://dl.acm.org/citation.cfm?id=3242155";>out-of-order data that 
may impact their business logic. In Kafka Streams, there are two causes that 
could potentially
 result in out-of-order data arrivals with respect to their timestamps:
 
 



[kafka] branch trunk updated: KAFKA-8208: Change paper link directly to ASM (#6572)

2019-04-13 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 312e55b  KAFKA-8208: Change paper link directly to ASM (#6572)
312e55b is described below

commit 312e55bf56c4670f0a2701b94dc21bfc8fa2990d
Author: Bill Bejeck 
AuthorDate: Sat Apr 13 18:59:40 2019 -0400

KAFKA-8208: Change paper link directly to ASM (#6572)

Reviewers: Matthias J. Sax , Victoria Bialas 

---
 docs/streams/core-concepts.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html
index 79f5b82..1e1aeb7 100644
--- a/docs/streams/core-concepts.html
+++ b/docs/streams/core-concepts.html
@@ -224,7 +224,7 @@
 
 
 Besides the guarantee that each record will be processed exactly-once, 
another issue that many stream processing application will face is how to
-handle https://www.confluent.io/wp-content/uploads/streams-tables-two-sides-same-coin.pdf";>out-of-order
 data that may impact their business logic. In Kafka Streams, there are two 
causes that could potentially
+handle https://dl.acm.org/citation.cfm?id=3242155";>out-of-order data that 
may impact their business logic. In Kafka Streams, there are two causes that 
could potentially
 result in out-of-order data arrivals with respect to their timestamps:
 
 



[kafka] branch 2.2 updated: KAFKA-8208: Change paper link directly to ASM (#6572)

2019-04-16 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new b73c077  KAFKA-8208: Change paper link directly to ASM (#6572)
b73c077 is described below

commit b73c07759f8325f20ca6d7df232b4a6b4acbba9b
Author: Bill Bejeck 
AuthorDate: Sat Apr 13 18:59:40 2019 -0400

KAFKA-8208: Change paper link directly to ASM (#6572)

Reviewers: Matthias J. Sax , Victoria Bialas 

---
 docs/streams/core-concepts.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html
index c925c2e..391167d 100644
--- a/docs/streams/core-concepts.html
+++ b/docs/streams/core-concepts.html
@@ -224,7 +224,7 @@
 
 
 Besides the guarantee that each record will be processed exactly-once, 
another issue that many stream processing application will face is how to
-handle https://www.confluent.io/wp-content/uploads/streams-tables-two-sides-same-coin.pdf";>out-of-order
 data that may impact their business logic. In Kafka Streams, there are two 
causes that could potentially
+handle https://dl.acm.org/citation.cfm?id=3242155";>out-of-order data that 
may impact their business logic. In Kafka Streams, there are two causes that 
could potentially
 result in out-of-order data arrivals with respect to their timestamps:
 
 



[kafka] branch 2.1 updated: KAFKA-8208: Change paper link directly to ASM (#6572)

2019-04-16 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.1 by this push:
 new f954a56  KAFKA-8208: Change paper link directly to ASM (#6572)
f954a56 is described below

commit f954a5663bb23fcc1486e5c574ca4e364387a291
Author: Bill Bejeck 
AuthorDate: Sat Apr 13 18:59:40 2019 -0400

KAFKA-8208: Change paper link directly to ASM (#6572)

Reviewers: Matthias J. Sax , Victoria Bialas 

---
 docs/streams/core-concepts.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html
index c925c2e..391167d 100644
--- a/docs/streams/core-concepts.html
+++ b/docs/streams/core-concepts.html
@@ -224,7 +224,7 @@
 
 
 Besides the guarantee that each record will be processed exactly-once, 
another issue that many stream processing application will face is how to
-handle https://www.confluent.io/wp-content/uploads/streams-tables-two-sides-same-coin.pdf";>out-of-order
 data that may impact their business logic. In Kafka Streams, there are two 
causes that could potentially
+handle https://dl.acm.org/citation.cfm?id=3242155";>out-of-order data that 
may impact their business logic. In Kafka Streams, there are two causes that 
could potentially
 result in out-of-order data arrivals with respect to their timestamps:
 
 



[kafka] branch 2.2 updated: KAFKA-8209: Wrong link for KStreams DSL in core concepts doc (#6564)

2019-04-16 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new b83ab55  KAFKA-8209: Wrong link for KStreams DSL in core concepts doc 
(#6564)
b83ab55 is described below

commit b83ab55fee72fc50a2c2d99c585563e12b9ff0db
Author: Bill Bejeck 
AuthorDate: Sat Apr 13 18:50:27 2019 -0400

KAFKA-8209: Wrong link for KStreams DSL in core concepts doc (#6564)

Reviewers Matthias J. Sax , Michael Drogalis 
, Victoria Bialas 
---
 docs/streams/core-concepts.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html
index 391167d..1e1aeb7 100644
--- a/docs/streams/core-concepts.html
+++ b/docs/streams/core-concepts.html
@@ -185,7 +185,7 @@
 Some stream processing applications don't require state, which means 
the processing of a message is independent from
 the processing of all other messages.
 However, being able to maintain state opens up many possibilities for 
sophisticated stream processing applications: you
-can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
+can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
 
 
 Kafka Streams provides so-called state stores, which can be 
used by stream processing applications to store and query data.



[kafka] branch 2.1 updated: KAFKA-8209: Wrong link for KStreams DSL in core concepts doc (#6564)

2019-04-16 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.1 by this push:
 new fc151ff  KAFKA-8209: Wrong link for KStreams DSL in core concepts doc 
(#6564)
fc151ff is described below

commit fc151ffd33761326ed1ab16d735195baf16b2e8a
Author: Bill Bejeck 
AuthorDate: Sat Apr 13 18:50:27 2019 -0400

KAFKA-8209: Wrong link for KStreams DSL in core concepts doc (#6564)

Reviewers Matthias J. Sax , Michael Drogalis 
, Victoria Bialas 
---
 docs/streams/core-concepts.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html
index 391167d..1e1aeb7 100644
--- a/docs/streams/core-concepts.html
+++ b/docs/streams/core-concepts.html
@@ -185,7 +185,7 @@
 Some stream processing applications don't require state, which means 
the processing of a message is independent from
 the processing of all other messages.
 However, being able to maintain state opens up many possibilities for 
sophisticated stream processing applications: you
-can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
+can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
 
 
 Kafka Streams provides so-called state stores, which can be 
used by stream processing applications to store and query data.



[kafka] branch 2.0 updated: KAFKA-8209: Wrong link for KStreams DSL in core concepts doc (#6564)

2019-04-16 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.0
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.0 by this push:
 new 60cc56d  KAFKA-8209: Wrong link for KStreams DSL in core concepts doc 
(#6564)
60cc56d is described below

commit 60cc56d3263558f59b3068f931a3d103ce78d5c3
Author: Bill Bejeck 
AuthorDate: Sat Apr 13 18:50:27 2019 -0400

KAFKA-8209: Wrong link for KStreams DSL in core concepts doc (#6564)

Reviewers Matthias J. Sax , Michael Drogalis 
, Victoria Bialas 
---
 docs/streams/core-concepts.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html
index b6d7762..594efaa 100644
--- a/docs/streams/core-concepts.html
+++ b/docs/streams/core-concepts.html
@@ -185,7 +185,7 @@
 Some stream processing applications don't require state, which means 
the processing of a message is independent from
 the processing of all other messages.
 However, being able to maintain state opens up many possibilities for 
sophisticated stream processing applications: you
-can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
+can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
 
 
 Kafka Streams provides so-called state stores, which can be 
used by stream processing applications to store and query data.



[kafka] branch 1.1 updated: KAFKA-8209: Wrong link for KStreams DSL in core concepts doc (#6564)

2019-04-16 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 1.1
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/1.1 by this push:
 new d7aa7b8  KAFKA-8209: Wrong link for KStreams DSL in core concepts doc 
(#6564)
d7aa7b8 is described below

commit d7aa7b80e1aefc4d6940af3dec277e5cbd11e1a4
Author: Bill Bejeck 
AuthorDate: Sat Apr 13 18:50:27 2019 -0400

KAFKA-8209: Wrong link for KStreams DSL in core concepts doc (#6564)

Reviewers Matthias J. Sax , Michael Drogalis 
, Victoria Bialas 
---
 docs/streams/core-concepts.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html
index 0b0f43b..0f682de 100644
--- a/docs/streams/core-concepts.html
+++ b/docs/streams/core-concepts.html
@@ -133,7 +133,7 @@
 Some stream processing applications don't require state, which means 
the processing of a message is independent from
 the processing of all other messages.
 However, being able to maintain state opens up many possibilities for 
sophisticated stream processing applications: you
-can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
+can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
 
 
 Kafka Streams provides so-called state stores, which can be 
used by stream processing applications to store and query data.



[kafka] branch 1.0 updated: KAFKA-8209: Wrong link for KStreams DSL in core concepts doc (#6564)

2019-04-16 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 1.0
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/1.0 by this push:
 new 2ffb62f  KAFKA-8209: Wrong link for KStreams DSL in core concepts doc 
(#6564)
2ffb62f is described below

commit 2ffb62f75c7d925d3477a9a1c04120a2a5070530
Author: Bill Bejeck 
AuthorDate: Sat Apr 13 18:50:27 2019 -0400

KAFKA-8209: Wrong link for KStreams DSL in core concepts doc (#6564)

Reviewers Matthias J. Sax , Michael Drogalis 
, Victoria Bialas 
---
 docs/streams/core-concepts.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html
index f2f32ad..e48e6f3 100644
--- a/docs/streams/core-concepts.html
+++ b/docs/streams/core-concepts.html
@@ -133,7 +133,7 @@
 Some stream processing applications don't require state, which means 
the processing of a message is independent from
 the processing of all other messages.
 However, being able to maintain state opens up many possibilities for 
sophisticated stream processing applications: you
-can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
+can join input streams, or group and aggregate data records. Many such 
stateful operators are provided by the Kafka 
Streams DSL.
 
 
 Kafka Streams provides so-called state stores, which can be 
used by stream processing applications to store and query data.



[kafka] branch 2.2 updated: KAFKA-8210: Fix link for streams table duality (#6573)

2019-04-16 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new 1ecf515  KAFKA-8210: Fix link for streams table duality (#6573)
1ecf515 is described below

commit 1ecf5155e9b6e9b584b4be0466a8923218095a19
Author: Bill Bejeck 
AuthorDate: Sat Apr 13 18:54:06 2019 -0400

KAFKA-8210: Fix link for streams table duality (#6573)

Reviewers: Victoria Bialas 
---
 docs/streams/developer-guide/dsl-api.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/streams/developer-guide/dsl-api.html 
b/docs/streams/developer-guide/dsl-api.html
index 0ddcbc5..f5c3df9 100644
--- a/docs/streams/developer-guide/dsl-api.html
+++ b/docs/streams/developer-guide/dsl-api.html
@@ -163,7 +163,7 @@
 
 
 
-We have already seen an example of a changelog stream in the 
section streams_concepts_duality. Another example are change 
data capture (CDC) records in the changelog of a relational database, 
representing which row in a database table was inserted, updated, or deleted.
+We have already seen an example of a changelog stream in the 
section streams and tables. Another example are change data capture 
(CDC) records in the changelog of a relational database, representing which row 
in a database table was inserted, updated, or deleted.
 
 
 



[kafka] branch 2.1 updated: KAFKA-8210: Fix link for streams table duality (#6573)

2019-04-16 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.1 by this push:
 new cf32c1b  KAFKA-8210: Fix link for streams table duality (#6573)
cf32c1b is described below

commit cf32c1b389cf2d68ef80abbe36a6fc3ebbd1a81c
Author: Bill Bejeck 
AuthorDate: Sat Apr 13 18:54:06 2019 -0400

KAFKA-8210: Fix link for streams table duality (#6573)

Reviewers: Victoria Bialas 
---
 docs/streams/developer-guide/dsl-api.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/streams/developer-guide/dsl-api.html 
b/docs/streams/developer-guide/dsl-api.html
index 9a6c44d..1012ea7 100644
--- a/docs/streams/developer-guide/dsl-api.html
+++ b/docs/streams/developer-guide/dsl-api.html
@@ -163,7 +163,7 @@
 
 
 
-We have already seen an example of a changelog stream in the 
section streams_concepts_duality. Another example are change 
data capture (CDC) records in the changelog of a relational database, 
representing which row in a database table was inserted, updated, or deleted.
+We have already seen an example of a changelog stream in the 
section streams and tables. Another example are change data capture 
(CDC) records in the changelog of a relational database, representing which row 
in a database table was inserted, updated, or deleted.
 
 
 



[kafka] branch 2.0 updated: KAFKA-8210: Fix link for streams table duality (#6573)

2019-04-16 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.0
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.0 by this push:
 new 839541e  KAFKA-8210: Fix link for streams table duality (#6573)
839541e is described below

commit 839541e4b96e2a031baecaeb72b12a0c56a1b8dd
Author: Bill Bejeck 
AuthorDate: Sat Apr 13 18:54:06 2019 -0400

KAFKA-8210: Fix link for streams table duality (#6573)

Reviewers: Victoria Bialas 
---
 docs/streams/developer-guide/dsl-api.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/streams/developer-guide/dsl-api.html 
b/docs/streams/developer-guide/dsl-api.html
index 1456b2c..49af92a 100644
--- a/docs/streams/developer-guide/dsl-api.html
+++ b/docs/streams/developer-guide/dsl-api.html
@@ -161,7 +161,7 @@
 
 
 
-We have already seen an example of a changelog stream in the 
section streams_concepts_duality. Another example are change 
data capture (CDC) records in the changelog of a relational database, 
representing which row in a database table was inserted, updated, or deleted.
+We have already seen an example of a changelog stream in the 
section streams and tables. Another example are change data capture 
(CDC) records in the changelog of a relational database, representing which row 
in a database table was inserted, updated, or deleted.
 
 
 



[kafka] branch trunk updated: KAFKA-7875: Add KStream.flatTransformValues (#6424)

2019-04-16 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 05668e9  KAFKA-7875: Add KStream.flatTransformValues (#6424)
05668e9 is described below

commit 05668e98f531cf4d6ddb0696f0f72675ca128581
Author: cadonna 
AuthorDate: Tue Apr 16 09:10:38 2019 -0700

KAFKA-7875: Add KStream.flatTransformValues (#6424)

Adds flatTrasformValues methods in KStream
Adds processor supplier and processor for flatTransformValues
Improves API documentation of transformValues

Reviewers: Matthias J. Sax ,  John Roesler 
, Bill Bejeck 
---
 .../org/apache/kafka/streams/kstream/KStream.java  | 225 +--
 .../internals/KStreamFlatTransformValues.java  |  70 ++
 .../streams/kstream/internals/KStreamImpl.java |  39 +++-
 .../KStreamTransformIntegrationTest.java   | 241 +
 .../internals/KStreamFlatTransformValuesTest.java  | 135 
 .../streams/kstream/internals/KStreamImplTest.java |  34 ++-
 6 files changed, 679 insertions(+), 65 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java 
b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
index 7faba82..8375336 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
@@ -263,6 +263,8 @@ public interface KStream {
  * @see #flatTransform(TransformerSupplier, String...)
  * @see #transformValues(ValueTransformerSupplier, String...)
  * @see #transformValues(ValueTransformerWithKeySupplier, String...)
+ * @see #flatTransformValues(ValueTransformerSupplier, String...)
+ * @see #flatTransformValues(ValueTransformerWithKeySupplier, String...)
  */
  KStream flatMap(final KeyValueMapper>> mapper);
 
@@ -304,6 +306,8 @@ public interface KStream {
  * @see #flatTransform(TransformerSupplier, String...)
  * @see #transformValues(ValueTransformerSupplier, String...)
  * @see #transformValues(ValueTransformerWithKeySupplier, String...)
+ * @see #flatTransformValues(ValueTransformerSupplier, String...)
+ * @see #flatTransformValues(ValueTransformerWithKeySupplier, String...)
  */
  KStream flatMapValues(final ValueMapper> mapper);
 
@@ -351,6 +355,8 @@ public interface KStream {
  * @see #flatTransform(TransformerSupplier, String...)
  * @see #transformValues(ValueTransformerSupplier, String...)
  * @see #transformValues(ValueTransformerWithKeySupplier, String...)
+ * @see #flatTransformValues(ValueTransformerSupplier, String...)
+ * @see #flatTransformValues(ValueTransformerWithKeySupplier, String...)
  */
  KStream flatMapValues(final ValueMapperWithKey> mapper);
 
@@ -627,7 +633,7 @@ public interface KStream {
  * Iterable transform(K key, V value) {
  * // can access this.state
  * List result = new ArrayList<>();
- * for (int i = 0; i < n; i++) {
+ * for (int i = 0; i < 3; i++) {
  * result.add(new KeyValue(key, value));
  * }
  * return result; // emits a list of key-value pairs via 
return
@@ -672,7 +678,7 @@ public interface KStream {
final String... stateStoreNames);
 
 /**
- * Transform the value of each input record into a new value (with 
possible new type) of the output record.
+ * Transform the value of each input record into a new value (with 
possibly a new type) of the output record.
  * A {@link ValueTransformer} (provided by the given {@link 
ValueTransformerSupplier}) is applied to each input
  * record value and computes a new value for it.
  * Thus, an input record {@code } can be transformed into an output 
record {@code }.
@@ -680,8 +686,8 @@ public interface KStream {
  * Furthermore, via {@link 
org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing 
progress
  * can be observed and additional periodic actions can be performed.
  * 
- * In order to assign a state, the state must be created and registered 
beforehand (it's not required to connect
- * global state stores; read-only access to global state stores is 
available by default):
+ * In order to assign a state store, the state store must be created and 
registered beforehand (it's not required to
+ * connect global state stores; read-only access to global state stores is 
available by default):
  * {@code
  * // create store
  * StoreBuilder> keyValueStoreBuilder =
@@ -693,12 +699,16 @@ public interface KStream {
  *
  * KStream outputStream = inputStream.transfor

[kafka] branch 1.1 updated: MINOR: Removed unused import (#6587)

2019-04-17 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 1.1
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/1.1 by this push:
 new 4b93e09  MINOR: Removed unused import (#6587)
4b93e09 is described below

commit 4b93e09a6e6145b54568fa0ca654f5fe85e32c16
Author: Bill Bejeck 
AuthorDate: Wed Apr 17 16:28:48 2019 -0400

MINOR: Removed unused import (#6587)

Reviewers:  Matthias J. Sax 
---
 streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java| 1 -
 .../src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java   | 1 -
 2 files changed, 2 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java 
b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
index f47e0dd..1ee775b 100644
--- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
+++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
@@ -47,7 +47,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Locale;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Set;
 
 import static org.apache.kafka.common.config.ConfigDef.Range.atLeast;
diff --git 
a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java
 
b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java
index e874da1..9e41c8a 100644
--- 
a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java
+++ 
b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java
@@ -46,7 +46,6 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
-import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;



[kafka] branch trunk updated: [HOT FIX] Check for null before deserializing in MeteredSessionStore (#6575)

2019-04-17 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new c783630  [HOT FIX] Check for null before deserializing in 
MeteredSessionStore  (#6575)
c783630 is described below

commit c7836307c3588ed5d267f32eabcd7dfc0dbeec80
Author: A. Sophie Blee-Goldman 
AuthorDate: Wed Apr 17 16:21:59 2019 -0700

[HOT FIX] Check for null before deserializing in MeteredSessionStore  
(#6575)

The fetchSession() method of SessionStore searches for a (single) specific 
session and returns null if none are found. This is analogous to fetch(key, 
time) in WindowStore or get(key) in KeyValueStore. MeteredWindowStore and 
MeteredKeyValueStore both check for a null result before attempting to 
deserialize, however MeteredSessionStore just blindly deserializes and as a 
result NPE is thrown when we search for a record that does not exist.

Reviewers: Guozhang Wang , Bill Bejeck 
, Bruno Cadonna 
---
 .../state/internals/MeteredSessionStore.java   | 23 --
 .../state/internals/MeteredKeyValueStoreTest.java  |  9 +
 .../state/internals/MeteredSessionStoreTest.java   | 13 ++--
 .../state/internals/MeteredWindowStoreTest.java|  2 +-
 4 files changed, 38 insertions(+), 9 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java
index 4631601..94b004e 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java
@@ -151,22 +151,28 @@ public class MeteredSessionStore
 @Override
 public V fetchSession(final K key, final long startTime, final long 
endTime) {
 Objects.requireNonNull(key, "key cannot be null");
-final V value;
 final Bytes bytesKey = keyBytes(key);
 final long startNs = time.nanoseconds();
 try {
-value = serdes.valueFrom(wrapped().fetchSession(bytesKey, 
startTime, endTime));
+final byte[] result = wrapped().fetchSession(bytesKey, startTime, 
endTime);
+if (result == null) {
+return null;
+}
+return serdes.valueFrom(result);
 } finally {
 metrics.recordLatency(flushTime, startNs, time.nanoseconds());
 }
-
-return value;
 }
 
 @Override
 public KeyValueIterator, V> fetch(final K key) {
 Objects.requireNonNull(key, "key cannot be null");
-return findSessions(key, 0, Long.MAX_VALUE);
+return new MeteredWindowedKeyValueIterator<>(
+wrapped().fetch(keyBytes(key)),
+fetchTime,
+metrics,
+serdes,
+time);
 }
 
 @Override
@@ -174,7 +180,12 @@ public class MeteredSessionStore
   final K to) {
 Objects.requireNonNull(from, "from cannot be null");
 Objects.requireNonNull(to, "to cannot be null");
-return findSessions(from, to, 0, Long.MAX_VALUE);
+return new MeteredWindowedKeyValueIterator<>(
+wrapped().fetch(keyBytes(from), keyBytes(to)),
+fetchTime,
+metrics,
+serdes,
+time);
 }
 
 @Override
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java
index b8fc88e..5cbe95c 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java
@@ -55,6 +55,7 @@ import static org.easymock.EasyMock.verify;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 @RunWith(EasyMockRunner.class)
@@ -244,6 +245,14 @@ public class MeteredKeyValueStoreTest {
 }
 
 @Test
+public void shouldNotThrowNullPointerExceptionIfGetReturnsNull() {
+expect(inner.get(Bytes.wrap("a".getBytes(.andReturn(null);
+
+init();
+assertNull(metered.get("a"));
+}
+
+@Test
 public void shouldNotSetFlushListenerOnWrappedNoneCachingStore() {
 assertFalse(metered.setFlushListener(null, false));
 }
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java
 
b/streams/src/test/java/org/apache/kafka/

[kafka] branch trunk updated: KAFKA-6958: Allow to name operation using parameter classes (#6410)

2019-04-18 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 075b368  KAFKA-6958: Allow to name operation using parameter classes 
(#6410)
075b368 is described below

commit 075b368d47ad3144518e4d425f9155e35f15f418
Author: Florian Hussonnois 
AuthorDate: Fri Apr 19 00:45:34 2019 +0200

KAFKA-6958: Allow to name operation using parameter classes (#6410)

This is the 2nd PR for the KIP-307
Reviewers: Matthias J. Sax ,  John Roesler 
, Bill Bejeck 
---
 .../org/apache/kafka/streams/StreamsBuilder.java   |  16 +++-
 .../org/apache/kafka/streams/kstream/Consumed.java |  46 +++--
 .../org/apache/kafka/streams/kstream/Grouped.java  |   7 +-
 .../apache/kafka/streams/kstream/Materialized.java |   3 +-
 .../org/apache/kafka/streams/kstream/Named.java|  83 
 .../org/apache/kafka/streams/kstream/Printed.java  |  16 +++-
 .../org/apache/kafka/streams/kstream/Produced.java |  36 +--
 .../kstream/internals/ConsumedInternal.java|   5 +
 .../kstream/internals/InternalNameProvider.java|   4 +-
 .../kstream/internals/InternalStreamsBuilder.java  |   9 +-
 .../streams/kstream/internals/KStreamImpl.java |  16 ++--
 .../streams/kstream/internals/NamedInternal.java   |  81 
 .../streams/kstream/internals/PrintedInternal.java |   4 +
 .../kstream/internals/ProducedInternal.java|   6 ++
 .../internals/suppress/SuppressedInternal.java |  13 +--
 .../apache/kafka/streams/StreamsBuilderTest.java   | 105 ++---
 .../kafka/streams/kstream/MaterializedTest.java|   4 +-
 .../apache/kafka/streams/kstream/NamedTest.java|  48 ++
 .../kstream/internals/KGroupedStreamImplTest.java  |  14 +--
 .../kstream/internals/KGroupedTableImplTest.java   |   6 +-
 .../kstream/internals/NamedInternalTest.java   |  66 +
 21 files changed, 522 insertions(+), 66 deletions(-)

diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java 
b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java
index 9e89d7a..abb5aa1 100644
--- a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java
+++ b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java
@@ -229,8 +229,10 @@ public class StreamsBuilder {
 Objects.requireNonNull(materialized, "materialized can't be null");
 final ConsumedInternal consumedInternal = new 
ConsumedInternal<>(consumed);
 
materialized.withKeySerde(consumedInternal.keySerde()).withValueSerde(consumedInternal.valueSerde());
+
 final MaterializedInternal> 
materializedInternal =
 new MaterializedInternal<>(materialized, internalStreamsBuilder, 
topic + "-");
+
 return internalStreamsBuilder.table(topic, consumedInternal, 
materializedInternal);
 }
 
@@ -280,8 +282,12 @@ public class StreamsBuilder {
 Objects.requireNonNull(topic, "topic can't be null");
 Objects.requireNonNull(consumed, "consumed can't be null");
 final ConsumedInternal consumedInternal = new 
ConsumedInternal<>(consumed);
+
 final MaterializedInternal> 
materializedInternal =
-new 
MaterializedInternal<>(Materialized.with(consumedInternal.keySerde(), 
consumedInternal.valueSerde()), internalStreamsBuilder, topic + "-");
+new MaterializedInternal<>(
+Materialized.with(consumedInternal.keySerde(), 
consumedInternal.valueSerde()),
+internalStreamsBuilder, topic + "-");
+
 return internalStreamsBuilder.table(topic, consumedInternal, 
materializedInternal);
 }
 
@@ -307,8 +313,10 @@ public class StreamsBuilder {
   final Materialized> materialized) {
 Objects.requireNonNull(topic, "topic can't be null");
 Objects.requireNonNull(materialized, "materialized can't be null");
+
 final MaterializedInternal> 
materializedInternal =
 new MaterializedInternal<>(materialized, internalStreamsBuilder, 
topic + "-");
+
 final ConsumedInternal consumedInternal =
 new 
ConsumedInternal<>(Consumed.with(materializedInternal.keySerde(), 
materializedInternal.valueSerde()));
 
@@ -336,8 +344,11 @@ public class StreamsBuilder {
 Objects.requireNonNull(topic, "topic can't be null");
 Objects.requireNonNull(consumed, "consumed can't be null");
 final ConsumedInternal consumedInternal = new 
ConsumedInternal<>(consumed);
+
 final MaterializedInternal> 
materializedInternal =
-new 
Material

[kafka] branch trunk updated: KAFKA-7895: fix Suppress changelog restore (#6536)

2019-04-20 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 6538e9e  KAFKA-7895: fix Suppress changelog restore (#6536)
6538e9e is described below

commit 6538e9e4d6c1f64fe3045a5c3fbfe306277a1bee
Author: John Roesler 
AuthorDate: Sat Apr 20 10:08:32 2019 -0500

KAFKA-7895: fix Suppress changelog restore (#6536)

Several issues have come to light since the 2.2.0 release:
upon restore, suppress incorrectly set the record metadata using the 
changelog record, instead of preserving the original metadata
restoring a tombstone incorrectly didn't update the buffer size and 
min-timestamp

Reviewers: Guozhang Wang , Matthias J. Sax 
,  Bruno Cadonna ,  Bill Bejeck 

---
 .../kafka/clients/consumer/ConsumerRecord.java |   6 +-
 .../suppress/KTableSuppressProcessor.java  |   4 +-
 .../internals/ProcessorRecordContext.java  | 137 -
 .../streams/state/internals/ContextualRecord.java  |  43 +-
 .../InMemoryTimeOrderedKeyValueBuffer.java | 173 --
 .../apache/kafka/streams/KeyValueTimestamp.java|  17 +
 .../SuppressionDurabilityIntegrationTest.java  | 153 --
 .../integration/SuppressionIntegrationTest.java|   3 +-
 .../integration/utils/IntegrationTestUtils.java|  60 +-
 .../suppress/KTableSuppressProcessorTest.java  |   1 -
 .../internals/TimeOrderedKeyValueBufferTest.java   | 604 +
 .../kafka/test/MockInternalProcessorContext.java   | 104 
 12 files changed, 1195 insertions(+), 110 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java 
b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
index 0413d5b..a7dad7b 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
@@ -157,6 +157,8 @@ public class ConsumerRecord {
   Optional leaderEpoch) {
 if (topic == null)
 throw new IllegalArgumentException("Topic cannot be null");
+if (headers == null)
+throw new IllegalArgumentException("Headers cannot be null");
 
 this.topic = topic;
 this.partition = partition;
@@ -173,7 +175,7 @@ public class ConsumerRecord {
 }
 
 /**
- * The topic this record is received from
+ * The topic this record is received from (never null)
  */
 public String topic() {
 return this.topic;
@@ -187,7 +189,7 @@ public class ConsumerRecord {
 }
 
 /**
- * The headers
+ * The headers (never null)
  */
 public Headers headers() {
 return headers;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessor.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessor.java
index 691e09e..686002a 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessor.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessor.java
@@ -33,8 +33,6 @@ import 
org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
 import org.apache.kafka.streams.state.internals.ContextualRecord;
 import org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBuffer;
 
-import java.util.Objects;
-
 import static java.util.Objects.requireNonNull;
 
 public class KTableSuppressProcessor implements Processor> {
@@ -78,7 +76,7 @@ public class KTableSuppressProcessor implements 
Processor> {
 
 keySerde = keySerde == null ? (Serde) context.keySerde() : keySerde;
 valueSerde = valueSerde == null ? 
FullChangeSerde.castOrWrap(context.valueSerde()) : valueSerde;
-buffer = Objects.requireNonNull((TimeOrderedKeyValueBuffer) 
context.getStateStore(storeName));
+buffer = requireNonNull((TimeOrderedKeyValueBuffer) 
context.getStateStore(storeName));
 }
 
 @Override
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java
index 0001274..aacb801 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java
@@ -18,10 +18,15 @@ package org.apache.kafka.streams.processor.internals;
 
 import org.apache.kafka.common.header.Header;
 import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeader;
+import org.apache.kafka.common.header.i

[kafka] branch trunk updated: MINOR: Move log statement stating producer closed after possible exception (#6606)

2019-04-22 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new a552a27  MINOR: Move log statement stating producer closed after 
possible exception (#6606)
a552a27 is described below

commit a552a2704e36b6db0a6295370d4e190ff212d5d1
Author: Bill Bejeck 
AuthorDate: Mon Apr 22 18:59:00 2019 -0400

MINOR: Move log statement stating producer closed after possible exception 
(#6606)

In the KafkaProducer#close method we have a debug log statement Kafka 
producer has been closed then a few lines later a KafkaException can occur.
This could be confusing to users, so this PR simply moves the log statement 
to after the possible exception to avoid confusing information in the logs.

Reviewers:  Guozhang Wang 
---
 .../src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
index e9d2626..83a4d51 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
@@ -1189,7 +1189,6 @@ public class KafkaProducer implements Producer {
 ClientUtils.closeQuietly(valueSerializer, "producer valueSerializer", 
firstException);
 ClientUtils.closeQuietly(partitioner, "producer partitioner", 
firstException);
 AppInfoParser.unregisterAppInfo(JMX_PREFIX, clientId, metrics);
-log.debug("Kafka producer has been closed");
 Throwable exception = firstException.get();
 if (exception != null && !swallowException) {
 if (exception instanceof InterruptException) {
@@ -1197,6 +1196,7 @@ public class KafkaProducer implements Producer {
 }
 throw new KafkaException("Failed to close kafka producer", 
exception);
 }
+log.debug("Kafka producer has been closed");
 }
 
 private static Map propsToMap(Properties properties) {



[kafka] branch 2.2 updated: KAFKA-7895: fix Suppress changelog restore (#6536) (#6615)

2019-04-22 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new 6c74506  KAFKA-7895: fix Suppress changelog restore (#6536) (#6615)
6c74506 is described below

commit 6c74506728af0b03c1aae167f6957efe4b128173
Author: John Roesler 
AuthorDate: Mon Apr 22 19:21:56 2019 -0500

KAFKA-7895: fix Suppress changelog restore (#6536) (#6615)

Cherry-picked from #6536 / 6538e9e

Reviewers: Bill Bejeck 
---
 .../kafka/clients/consumer/ConsumerRecord.java |   6 +-
 .../suppress/KTableSuppressProcessor.java  |   4 +-
 .../internals/ProcessorRecordContext.java  | 137 -
 .../streams/state/internals/ContextualRecord.java  |  43 +-
 .../InMemoryTimeOrderedKeyValueBuffer.java | 173 --
 .../apache/kafka/streams/KeyValueTimestamp.java|  17 +
 .../SuppressionDurabilityIntegrationTest.java  | 166 --
 .../integration/SuppressionIntegrationTest.java|   3 +-
 .../integration/utils/IntegrationTestUtils.java|  60 +-
 .../suppress/KTableSuppressProcessorTest.java  |   1 -
 .../internals/TimeOrderedKeyValueBufferTest.java   | 604 +
 .../kafka/test/MockInternalProcessorContext.java   | 104 +++-
 12 files changed, 1200 insertions(+), 118 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java 
b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
index 0413d5b..a7dad7b 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
@@ -157,6 +157,8 @@ public class ConsumerRecord {
   Optional leaderEpoch) {
 if (topic == null)
 throw new IllegalArgumentException("Topic cannot be null");
+if (headers == null)
+throw new IllegalArgumentException("Headers cannot be null");
 
 this.topic = topic;
 this.partition = partition;
@@ -173,7 +175,7 @@ public class ConsumerRecord {
 }
 
 /**
- * The topic this record is received from
+ * The topic this record is received from (never null)
  */
 public String topic() {
 return this.topic;
@@ -187,7 +189,7 @@ public class ConsumerRecord {
 }
 
 /**
- * The headers
+ * The headers (never null)
  */
 public Headers headers() {
 return headers;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessor.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessor.java
index 691e09e..686002a 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessor.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessor.java
@@ -33,8 +33,6 @@ import 
org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
 import org.apache.kafka.streams.state.internals.ContextualRecord;
 import org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBuffer;
 
-import java.util.Objects;
-
 import static java.util.Objects.requireNonNull;
 
 public class KTableSuppressProcessor implements Processor> {
@@ -78,7 +76,7 @@ public class KTableSuppressProcessor implements 
Processor> {
 
 keySerde = keySerde == null ? (Serde) context.keySerde() : keySerde;
 valueSerde = valueSerde == null ? 
FullChangeSerde.castOrWrap(context.valueSerde()) : valueSerde;
-buffer = Objects.requireNonNull((TimeOrderedKeyValueBuffer) 
context.getStateStore(storeName));
+buffer = requireNonNull((TimeOrderedKeyValueBuffer) 
context.getStateStore(storeName));
 }
 
 @Override
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java
index 0001274..aacb801 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java
@@ -18,10 +18,15 @@ package org.apache.kafka.streams.processor.internals;
 
 import org.apache.kafka.common.header.Header;
 import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeader;
+import org.apache.kafka.common.header.internals.RecordHeaders;
 import org.apache.kafka.streams.processor.RecordContext;
 
+import java.nio.ByteBuffer;
 import java.util.Objects;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+
 public class ProcessorRecordContext implements RecordContext {
 
 long timestamp;
@@ -80,13 +85,13 @@ public cl

[kafka] branch 2.1 updated: KAFKA-7895: fix Suppress changelog restore (#6536) (#6616)

2019-04-23 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.1 by this push:
 new a193f37  KAFKA-7895: fix Suppress changelog restore (#6536) (#6616)
a193f37 is described below

commit a193f370c0cd43d056dd913e97e457e636dbf76d
Author: John Roesler 
AuthorDate: Tue Apr 23 12:42:04 2019 -0500

KAFKA-7895: fix Suppress changelog restore (#6536) (#6616)

Several issues have come to light since the 2.2.0 release:
upon restore, suppress incorrectly set the record metadata using the 
changelog record, instead of preserving the original metadata
restoring a tombstone incorrectly didn't update the buffer size and 
min-timestamp

Cherry-picked from #6536 / 6538e9e

Reviewers: Bill Bejeck
---
 .../kafka/clients/consumer/ConsumerRecord.java |   6 +-
 .../suppress/KTableSuppressProcessor.java  |   4 +-
 .../internals/ProcessorRecordContext.java  | 137 -
 .../streams/state/internals/ContextualRecord.java  |  43 +-
 .../InMemoryTimeOrderedKeyValueBuffer.java | 173 --
 .../apache/kafka/streams/KeyValueTimestamp.java|  17 +
 .../SuppressionDurabilityIntegrationTest.java  | 166 --
 .../integration/SuppressionIntegrationTest.java|   3 +-
 .../integration/utils/IntegrationTestUtils.java|  60 +-
 .../suppress/KTableSuppressProcessorTest.java  |   1 -
 .../internals/TimeOrderedKeyValueBufferTest.java   | 604 +
 .../kafka/test/MockInternalProcessorContext.java   | 104 +++-
 12 files changed, 1200 insertions(+), 118 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java 
b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
index 0413d5b..a7dad7b 100644
--- 
a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
+++ 
b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
@@ -157,6 +157,8 @@ public class ConsumerRecord {
   Optional leaderEpoch) {
 if (topic == null)
 throw new IllegalArgumentException("Topic cannot be null");
+if (headers == null)
+throw new IllegalArgumentException("Headers cannot be null");
 
 this.topic = topic;
 this.partition = partition;
@@ -173,7 +175,7 @@ public class ConsumerRecord {
 }
 
 /**
- * The topic this record is received from
+ * The topic this record is received from (never null)
  */
 public String topic() {
 return this.topic;
@@ -187,7 +189,7 @@ public class ConsumerRecord {
 }
 
 /**
- * The headers
+ * The headers (never null)
  */
 public Headers headers() {
 return headers;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessor.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessor.java
index 4058083..63c 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessor.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessor.java
@@ -31,8 +31,6 @@ import 
org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
 import org.apache.kafka.streams.state.internals.ContextualRecord;
 import org.apache.kafka.streams.state.internals.TimeOrderedKeyValueBuffer;
 
-import java.util.Objects;
-
 import static java.util.Objects.requireNonNull;
 
 public class KTableSuppressProcessor implements Processor> {
@@ -74,7 +72,7 @@ public class KTableSuppressProcessor implements 
Processor> {
 internalProcessorContext = (InternalProcessorContext) context;
 keySerde = keySerde == null ? (Serde) context.keySerde() : keySerde;
 valueSerde = valueSerde == null ? 
FullChangeSerde.castOrWrap(context.valueSerde()) : valueSerde;
-buffer = Objects.requireNonNull((TimeOrderedKeyValueBuffer) 
context.getStateStore(storeName));
+buffer = requireNonNull((TimeOrderedKeyValueBuffer) 
context.getStateStore(storeName));
 }
 
 @Override
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java
index da44e96..4f991a2 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorRecordContext.java
@@ -18,10 +18,15 @@ package org.apache.kafka.streams.processor.internals;
 
 import org.apache.kafka.common.header.Header;
 import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.

[kafka] branch trunk updated: KAFKA-8227 DOCS Fixed missing links duality of streams tables (#6625)

2019-04-24 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new dd81314  KAFKA-8227 DOCS Fixed missing links duality of streams tables 
(#6625)
dd81314 is described below

commit dd8131499fe99300338b2238a994923ac94a698e
Author: Victoria Bialas 
AuthorDate: Wed Apr 24 14:54:29 2019 -0700

KAFKA-8227 DOCS Fixed missing links duality of streams tables (#6625)

Fixed missing links duality of streams tables

Reviewers: Jim Galasyn  Bill Bejeck 

---
 docs/streams/core-concepts.html | 39 +++
 1 file changed, 19 insertions(+), 20 deletions(-)

diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html
index 1e1aeb7..474cac9 100644
--- a/docs/streams/core-concepts.html
+++ b/docs/streams/core-concepts.html
@@ -63,7 +63,7 @@
 
 A stream is the most important abstraction provided by 
Kafka Streams: it represents an unbounded, continuously updating data set. A 
stream is an ordered, replayable, and fault-tolerant sequence of immutable data 
records, where a data record is defined as a key-value pair.
 A stream processing application is any program that makes 
use of the Kafka Streams library. It defines its computational logic through 
one or more processor topologies, where a processor topology is a graph 
of stream processors (nodes) that are connected by streams (edges).
-A stream processor is a node in the 
processor topology; it represents a processing step to transform data in 
streams by receiving one input record at a time from its upstream processors in 
the topology, applying its operation to it, and may subsequently produce one or 
more output records to its downstream processors. 
+A stream
 processor is a node in the processor topology; it represents a 
processing step to transform data in streams by receiving one input record at a 
time from its upstream processors in the topology, applying its operation to 
it, and may subsequently produce one or more output records to its downstream 
processors. 
 
 
 There are two special processors in the topology:
@@ -159,25 +159,24 @@
 
 
 
-Any stream processing technology must therefore provide 
first-class support for streams and tables.
-Kafka's Streams API provides such functionality through its core 
abstractions for
-streams 
<streams_concepts_kstream> and
-tables 
<streams_concepts_ktable>, which we will talk about in a minute.
-Now, an interesting observation is that there is actually a 
close relationship between streams and tables,
-the so-called stream-table duality.
-And Kafka exploits this duality in many ways: for example, to make 
your applications
-elastic 
<streams_developer-guide_execution-scaling>,
-to support fault-tolerant stateful processing 
<streams_developer-guide_state-store_fault-tolerance>,
-or to run interactive 
queries <streams_concepts_interactive-queries>
-against your application's latest processing results. And, beyond its 
internal usage, the Kafka Streams API
-also allows developers to exploit this duality in their own 
applications.
-
-
-
-Before we discuss concepts such as aggregations <streams_concepts_aggregations>
-in Kafka Streams we must first introduce tables in 
more detail, and talk about the aforementioned stream-table duality.
-Essentially, this duality means that a stream can be viewed as a 
table, and a table can be viewed as a stream.
-
+  Any stream processing technology must therefore provide 
first-class support for streams and tables.
+  Kafka's Streams API provides such functionality through its core 
abstractions for 
+  streams
+  and tables,
+  which we will talk about in a minute. Now, an interesting observation is 
that there is actually a close relationship between streams and 
tables,
+  the so-called stream-table duality. And Kafka exploits this duality in 
many ways: for example, to make your applications
+  elastic,
+  to support fault-tolerant
 stateful processing,
+  or to run interactive
 queries
+  against your application's latest processing results. And, beyond its 
internal usage, the Kafka Streams API
+  also allows developers to exploit this duality in their own applications.
+  
+
+  
+  Before we discuss concepts such as aggregations
+  in Kafka Streams, we must first introduce tables in 
more detail, and talk about the aforementioned stream-table duality.
+  Essentially, this duality means that a stream can be viewed as a table, 
and a table can be viewed as a stream.
+  
 
 States
 



[kafka] branch 2.2 updated: KAFKA-8227 DOCS Fixed missing links duality of streams tables (#6625)

2019-04-24 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new 0589810  KAFKA-8227 DOCS Fixed missing links duality of streams tables 
(#6625)
0589810 is described below

commit 0589810a9988b208d0b6b86fcda0a7d7c3c0f310
Author: Victoria Bialas 
AuthorDate: Wed Apr 24 14:54:29 2019 -0700

KAFKA-8227 DOCS Fixed missing links duality of streams tables (#6625)

Fixed missing links duality of streams tables

Reviewers: Jim Galasyn  Bill Bejeck 

---
 docs/streams/core-concepts.html | 39 +++
 1 file changed, 19 insertions(+), 20 deletions(-)

diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html
index 1e1aeb7..474cac9 100644
--- a/docs/streams/core-concepts.html
+++ b/docs/streams/core-concepts.html
@@ -63,7 +63,7 @@
 
 A stream is the most important abstraction provided by 
Kafka Streams: it represents an unbounded, continuously updating data set. A 
stream is an ordered, replayable, and fault-tolerant sequence of immutable data 
records, where a data record is defined as a key-value pair.
 A stream processing application is any program that makes 
use of the Kafka Streams library. It defines its computational logic through 
one or more processor topologies, where a processor topology is a graph 
of stream processors (nodes) that are connected by streams (edges).
-A stream processor is a node in the 
processor topology; it represents a processing step to transform data in 
streams by receiving one input record at a time from its upstream processors in 
the topology, applying its operation to it, and may subsequently produce one or 
more output records to its downstream processors. 
+A stream
 processor is a node in the processor topology; it represents a 
processing step to transform data in streams by receiving one input record at a 
time from its upstream processors in the topology, applying its operation to 
it, and may subsequently produce one or more output records to its downstream 
processors. 
 
 
 There are two special processors in the topology:
@@ -159,25 +159,24 @@
 
 
 
-Any stream processing technology must therefore provide 
first-class support for streams and tables.
-Kafka's Streams API provides such functionality through its core 
abstractions for
-streams 
<streams_concepts_kstream> and
-tables 
<streams_concepts_ktable>, which we will talk about in a minute.
-Now, an interesting observation is that there is actually a 
close relationship between streams and tables,
-the so-called stream-table duality.
-And Kafka exploits this duality in many ways: for example, to make 
your applications
-elastic 
<streams_developer-guide_execution-scaling>,
-to support fault-tolerant stateful processing 
<streams_developer-guide_state-store_fault-tolerance>,
-or to run interactive 
queries <streams_concepts_interactive-queries>
-against your application's latest processing results. And, beyond its 
internal usage, the Kafka Streams API
-also allows developers to exploit this duality in their own 
applications.
-
-
-
-Before we discuss concepts such as aggregations <streams_concepts_aggregations>
-in Kafka Streams we must first introduce tables in 
more detail, and talk about the aforementioned stream-table duality.
-Essentially, this duality means that a stream can be viewed as a 
table, and a table can be viewed as a stream.
-
+  Any stream processing technology must therefore provide 
first-class support for streams and tables.
+  Kafka's Streams API provides such functionality through its core 
abstractions for 
+  streams
+  and tables,
+  which we will talk about in a minute. Now, an interesting observation is 
that there is actually a close relationship between streams and 
tables,
+  the so-called stream-table duality. And Kafka exploits this duality in 
many ways: for example, to make your applications
+  elastic,
+  to support fault-tolerant
 stateful processing,
+  or to run interactive
 queries
+  against your application's latest processing results. And, beyond its 
internal usage, the Kafka Streams API
+  also allows developers to exploit this duality in their own applications.
+  
+
+  
+  Before we discuss concepts such as aggregations
+  in Kafka Streams, we must first introduce tables in 
more detail, and talk about the aforementioned stream-table duality.
+  Essentially, this duality means that a stream can be viewed as a table, 
and a table can be viewed as a stream.
+  
 
 States
 



[kafka] branch 2.1 updated: KAFKA-8227 DOCS Fixed missing links duality of streams tables (#6625)

2019-04-24 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.1 by this push:
 new 0fd930b  KAFKA-8227 DOCS Fixed missing links duality of streams tables 
(#6625)
0fd930b is described below

commit 0fd930bf826114d589e741e55a5753e2a2a87f58
Author: Victoria Bialas 
AuthorDate: Wed Apr 24 14:54:29 2019 -0700

KAFKA-8227 DOCS Fixed missing links duality of streams tables (#6625)

Fixed missing links duality of streams tables

Reviewers: Jim Galasyn  Bill Bejeck 

---
 docs/streams/core-concepts.html | 39 +++
 1 file changed, 19 insertions(+), 20 deletions(-)

diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html
index 1e1aeb7..474cac9 100644
--- a/docs/streams/core-concepts.html
+++ b/docs/streams/core-concepts.html
@@ -63,7 +63,7 @@
 
 A stream is the most important abstraction provided by 
Kafka Streams: it represents an unbounded, continuously updating data set. A 
stream is an ordered, replayable, and fault-tolerant sequence of immutable data 
records, where a data record is defined as a key-value pair.
 A stream processing application is any program that makes 
use of the Kafka Streams library. It defines its computational logic through 
one or more processor topologies, where a processor topology is a graph 
of stream processors (nodes) that are connected by streams (edges).
-A stream processor is a node in the 
processor topology; it represents a processing step to transform data in 
streams by receiving one input record at a time from its upstream processors in 
the topology, applying its operation to it, and may subsequently produce one or 
more output records to its downstream processors. 
+A stream
 processor is a node in the processor topology; it represents a 
processing step to transform data in streams by receiving one input record at a 
time from its upstream processors in the topology, applying its operation to 
it, and may subsequently produce one or more output records to its downstream 
processors. 
 
 
 There are two special processors in the topology:
@@ -159,25 +159,24 @@
 
 
 
-Any stream processing technology must therefore provide 
first-class support for streams and tables.
-Kafka's Streams API provides such functionality through its core 
abstractions for
-streams 
<streams_concepts_kstream> and
-tables 
<streams_concepts_ktable>, which we will talk about in a minute.
-Now, an interesting observation is that there is actually a 
close relationship between streams and tables,
-the so-called stream-table duality.
-And Kafka exploits this duality in many ways: for example, to make 
your applications
-elastic 
<streams_developer-guide_execution-scaling>,
-to support fault-tolerant stateful processing 
<streams_developer-guide_state-store_fault-tolerance>,
-or to run interactive 
queries <streams_concepts_interactive-queries>
-against your application's latest processing results. And, beyond its 
internal usage, the Kafka Streams API
-also allows developers to exploit this duality in their own 
applications.
-
-
-
-Before we discuss concepts such as aggregations <streams_concepts_aggregations>
-in Kafka Streams we must first introduce tables in 
more detail, and talk about the aforementioned stream-table duality.
-Essentially, this duality means that a stream can be viewed as a 
table, and a table can be viewed as a stream.
-
+  Any stream processing technology must therefore provide 
first-class support for streams and tables.
+  Kafka's Streams API provides such functionality through its core 
abstractions for 
+  streams
+  and tables,
+  which we will talk about in a minute. Now, an interesting observation is 
that there is actually a close relationship between streams and 
tables,
+  the so-called stream-table duality. And Kafka exploits this duality in 
many ways: for example, to make your applications
+  elastic,
+  to support fault-tolerant
 stateful processing,
+  or to run interactive
 queries
+  against your application's latest processing results. And, beyond its 
internal usage, the Kafka Streams API
+  also allows developers to exploit this duality in their own applications.
+  
+
+  
+  Before we discuss concepts such as aggregations
+  in Kafka Streams, we must first introduce tables in 
more detail, and talk about the aforementioned stream-table duality.
+  Essentially, this duality means that a stream can be viewed as a table, 
and a table can be viewed as a stream.
+  
 
 States
 



[kafka] branch 2.0 updated: KAFKA-8227 DOCS Fixed missing links duality of streams tables (#6625)

2019-04-24 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.0
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.0 by this push:
 new b642eec  KAFKA-8227 DOCS Fixed missing links duality of streams tables 
(#6625)
b642eec is described below

commit b642eecd753764c1021096083e4d5a7cb3f0379e
Author: Victoria Bialas 
AuthorDate: Wed Apr 24 14:54:29 2019 -0700

KAFKA-8227 DOCS Fixed missing links duality of streams tables (#6625)

Fixed missing links duality of streams tables

Reviewers: Jim Galasyn  Bill Bejeck 

---
 docs/streams/core-concepts.html | 39 +++
 1 file changed, 19 insertions(+), 20 deletions(-)

diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html
index 594efaa..4e5a7ae 100644
--- a/docs/streams/core-concepts.html
+++ b/docs/streams/core-concepts.html
@@ -63,7 +63,7 @@
 
 A stream is the most important abstraction provided by 
Kafka Streams: it represents an unbounded, continuously updating data set. A 
stream is an ordered, replayable, and fault-tolerant sequence of immutable data 
records, where a data record is defined as a key-value pair.
 A stream processing application is any program that makes 
use of the Kafka Streams library. It defines its computational logic through 
one or more processor topologies, where a processor topology is a graph 
of stream processors (nodes) that are connected by streams (edges).
-A stream processor is a node in the 
processor topology; it represents a processing step to transform data in 
streams by receiving one input record at a time from its upstream processors in 
the topology, applying its operation to it, and may subsequently produce one or 
more output records to its downstream processors. 
+A stream
 processor is a node in the processor topology; it represents a 
processing step to transform data in streams by receiving one input record at a 
time from its upstream processors in the topology, applying its operation to 
it, and may subsequently produce one or more output records to its downstream 
processors. 
 
 
 There are two special processors in the topology:
@@ -159,25 +159,24 @@
 
 
 
-Any stream processing technology must therefore provide 
first-class support for streams and tables.
-Kafka's Streams API provides such functionality through its core 
abstractions for
-streams 
<streams_concepts_kstream> and
-tables 
<streams_concepts_ktable>, which we will talk about in a minute.
-Now, an interesting observation is that there is actually a 
close relationship between streams and tables,
-the so-called stream-table duality.
-And Kafka exploits this duality in many ways: for example, to make 
your applications
-elastic 
<streams_developer-guide_execution-scaling>,
-to support fault-tolerant stateful processing 
<streams_developer-guide_state-store_fault-tolerance>,
-or to run interactive 
queries <streams_concepts_interactive-queries>
-against your application's latest processing results. And, beyond its 
internal usage, the Kafka Streams API
-also allows developers to exploit this duality in their own 
applications.
-
-
-
-Before we discuss concepts such as aggregations <streams_concepts_aggregations>
-in Kafka Streams we must first introduce tables in 
more detail, and talk about the aforementioned stream-table duality.
-Essentially, this duality means that a stream can be viewed as a 
table, and a table can be viewed as a stream.
-
+  Any stream processing technology must therefore provide 
first-class support for streams and tables.
+  Kafka's Streams API provides such functionality through its core 
abstractions for 
+  streams
+  and tables,
+  which we will talk about in a minute. Now, an interesting observation is 
that there is actually a close relationship between streams and 
tables,
+  the so-called stream-table duality. And Kafka exploits this duality in 
many ways: for example, to make your applications
+  elastic,
+  to support fault-tolerant
 stateful processing,
+  or to run interactive
 queries
+  against your application's latest processing results. And, beyond its 
internal usage, the Kafka Streams API
+  also allows developers to exploit this duality in their own applications.
+  
+
+  
+  Before we discuss concepts such as aggregations
+  in Kafka Streams, we must first introduce tables in 
more detail, and talk about the aforementioned stream-table duality.
+  Essentially, this duality means that a stream can be viewed as a table, 
and a table can be viewed as a stream.
+  
 
 States
 



[kafka-site] branch asf-site updated: KAFKA-SITE DOCS 8227 - Add missing links Core Concepts duality of streams tables (#204)

2019-04-24 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/kafka-site.git


The following commit(s) were added to refs/heads/asf-site by this push:
 new 00e2637  KAFKA-SITE DOCS 8227 - Add missing links Core Concepts 
duality of streams tables (#204)
00e2637 is described below

commit 00e263735fd6254230a552d53e2921405a8dea7e
Author: Victoria Bialas 
AuthorDate: Wed Apr 24 15:17:13 2019 -0700

KAFKA-SITE DOCS 8227 - Add missing links Core Concepts duality of streams 
tables (#204)

Add missing links Core Concepts duality of streams tables

Reviewers: Joel Hamill , Bill Bejeck 
---
 10/streams/core-concepts.html |  3 ++-
 10/streams/developer-guide/processor-api.html |  2 +-
 11/streams/core-concepts.html |  5 +++--
 11/streams/developer-guide/processor-api.html |  2 +-
 20/streams/core-concepts.html | 26 --
 21/streams/core-concepts.html | 25 -
 22/streams/core-concepts.html | 27 +--
 22/streams/developer-guide/running-app.html   |  3 ++-
 8 files changed, 50 insertions(+), 43 deletions(-)

diff --git a/10/streams/core-concepts.html b/10/streams/core-concepts.html
index 81bfdf6..13a8b3e 100644
--- a/10/streams/core-concepts.html
+++ b/10/streams/core-concepts.html
@@ -57,13 +57,14 @@
 
 We first summarize the key concepts of Kafka Streams.
 
+
 
 Stream Processing 
Topology
 
 
 A stream is the most important abstraction provided by 
Kafka Streams: it represents an unbounded, continuously updating data set. A 
stream is an ordered, replayable, and fault-tolerant sequence of immutable data 
records, where a data record is defined as a key-value pair.
 A stream processing application is any program that makes 
use of the Kafka Streams library. It defines its computational logic through 
one or more processor topologies, where a processor topology is a graph 
of stream processors (nodes) that are connected by streams (edges).
-A stream processor is 
a node in the processor topology; it represents a processing step to transform 
data in streams by receiving one input record at a time from its upstream 
processors in the topology, applying its operation to it, and may subsequently 
produce one or more output records to its downstream processors. 
+A stream
 processor is a node in the processor topology; it represents a 
processing step to transform data in streams by receiving one input record at a 
time from its upstream processors in the topology, applying its operation to 
it, and may subsequently produce one or more output records to its downstream 
processors. 
 
 
 There are two special processors in the topology:
diff --git a/10/streams/developer-guide/processor-api.html 
b/10/streams/developer-guide/processor-api.html
index fdf6c86..1e1df65 100644
--- a/10/streams/developer-guide/processor-api.html
+++ b/10/streams/developer-guide/processor-api.html
@@ -66,7 +66,7 @@
 
 
 Defining a Stream Processor
-A stream 
processor is a node in the processor topology that represents a 
single processing step.
+A stream processor is a node in the processor topology that 
represents a single processing step.
 With the Processor API, you can define arbitrary stream 
processors that processes one received record at a time, and connect
 these processors with their associated state stores to compose 
the processor topology.
 You can define a customized stream processor by implementing 
the Processor 
interface, which provides the process() API method.
diff --git a/11/streams/core-concepts.html b/11/streams/core-concepts.html
index 473a268..bd930de 100644
--- a/11/streams/core-concepts.html
+++ b/11/streams/core-concepts.html
@@ -57,13 +57,14 @@
 
 We first summarize the key concepts of Kafka Streams.
 
+
 
 Stream Processing 
Topology
-
+
 
 A stream is the most important abstraction provided by 
Kafka Streams: it represents an unbounded, continuously updating data set. A 
stream is an ordered, replayable, and fault-tolerant sequence of immutable data 
records, where a data record is defined as a key-value pair.
 A stream processing application is any program that makes 
use of the Kafka Streams library. It defines its computational logic through 
one or more processor topologies, where a processor topology is a graph 
of stream processors (nodes) that are connected by streams (edges).
-A stream processor is a node in the 
processor topology; it represents a processing step to transform data in 
streams by receiving one input record at a time from its upstream processors in 
the topology, applying its operation to it, and may

[kafka] branch trunk updated: KAFKA-8298: Fix possible concurrent modification exception (#6643)

2019-05-01 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new ba1fc21  KAFKA-8298: Fix possible concurrent modification exception 
(#6643)
ba1fc21 is described below

commit ba1fc21864bbd4c1374695890f98ff1fa2614504
Author: Bill Bejeck 
AuthorDate: Wed May 1 12:07:45 2019 -0400

KAFKA-8298: Fix possible concurrent modification exception (#6643)

When processing multiple key-changing operations during the optimization 
phase a ConcurrentModificationException is possible.

Reviewers: Guozhang Wang 
---
 .../kstream/internals/InternalStreamsBuilder.java  |  8 --
 .../kstream/internals/graph/StreamsGraphTest.java  | 33 ++
 2 files changed, 39 insertions(+), 2 deletions(-)

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 960b030..e7a7678 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
@@ -42,9 +42,11 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Objects;
 import java.util.PriorityQueue;
 import java.util.Properties;
@@ -311,8 +313,10 @@ public class InternalStreamsBuilder implements 
InternalNameProvider {
 @SuppressWarnings("unchecked")
 private void maybeOptimizeRepartitionOperations() {
 maybeUpdateKeyChangingRepartitionNodeMap();
+final Iterator>> entryIterator =  
keyChangingOperationsToOptimizableRepartitionNodes.entrySet().iterator();
 
-for (final Map.Entry> entry : 
keyChangingOperationsToOptimizableRepartitionNodes.entrySet()) {
+while (entryIterator.hasNext()) {
+final Map.Entry> entry = entryIterator.next();
 
 final StreamsGraphNode keyChangingNode = entry.getKey();
 
@@ -368,7 +372,7 @@ public class InternalStreamsBuilder implements 
InternalNameProvider {
 }
 
 keyChangingNode.addChild(optimizedSingleRepartition);
-
keyChangingOperationsToOptimizableRepartitionNodes.remove(entry.getKey());
+entryIterator.remove();
 }
 }
 
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
index bd43685..0fecaa2 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
@@ -17,15 +17,21 @@
 
 package org.apache.kafka.streams.kstream.internals.graph;
 
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsBuilder;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.kstream.Grouped;
 import org.apache.kafka.streams.kstream.JoinWindows;
 import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.streams.kstream.Produced;
 import org.apache.kafka.streams.kstream.TimeWindows;
 import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.junit.Test;
 
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -68,6 +74,33 @@ public class StreamsGraphTest {
 }
 
 @Test
+public void shouldBeAbleToProcessNestedMultipleKeyChangingNodes() {
+final Properties properties = new Properties();
+properties.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, 
"test-application");
+properties.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
"localhost:9092");
+properties.setProperty(StreamsConfig.TOPOLOGY_OPTIMIZATION, 
StreamsConfig.OPTIMIZE);
+
+final StreamsBuilder builder = new StreamsBuilder();
+final KStream inputStream = 
builder.stream("inputTopic");
+
+final KStream changedKeyStream = 
inputStream.selectKey((k, v) -> v.substring(0, 5));
+
+// first repartition
+changedKeyStream.groupByKey(Grouped.as("count-repartition"))
+.count(Materialized.as("count-store"))
+.toStream().to("count-topic", Produced.with(Serdes.String(), 
Serdes.Long()));
+
+// sec

[kafka] branch 2.2 updated: KAFKA-8298: Fix possible concurrent modification exception (#6643)

2019-05-01 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new 977257f  KAFKA-8298: Fix possible concurrent modification exception 
(#6643)
977257f is described below

commit 977257f544aebcb59fadf4e424d32cecd9191875
Author: Bill Bejeck 
AuthorDate: Wed May 1 12:07:45 2019 -0400

KAFKA-8298: Fix possible concurrent modification exception (#6643)

When processing multiple key-changing operations during the optimization 
phase a ConcurrentModificationException is possible.

Reviewers: Guozhang Wang 
---
 .../kstream/internals/InternalStreamsBuilder.java  |  8 --
 .../kstream/internals/graph/StreamsGraphTest.java  | 33 ++
 2 files changed, 39 insertions(+), 2 deletions(-)

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 e0983eb..bf36389 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
@@ -42,9 +42,11 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Objects;
 import java.util.PriorityQueue;
 import java.util.Properties;
@@ -299,8 +301,10 @@ public class InternalStreamsBuilder implements 
InternalNameProvider {
 @SuppressWarnings("unchecked")
 private void maybeOptimizeRepartitionOperations() {
 maybeUpdateKeyChangingRepartitionNodeMap();
+final Iterator>> entryIterator =  
keyChangingOperationsToOptimizableRepartitionNodes.entrySet().iterator();
 
-for (final Map.Entry> entry : 
keyChangingOperationsToOptimizableRepartitionNodes.entrySet()) {
+while (entryIterator.hasNext()) {
+final Map.Entry> entry = entryIterator.next();
 
 final StreamsGraphNode keyChangingNode = entry.getKey();
 
@@ -356,7 +360,7 @@ public class InternalStreamsBuilder implements 
InternalNameProvider {
 }
 
 keyChangingNode.addChild(optimizedSingleRepartition);
-
keyChangingOperationsToOptimizableRepartitionNodes.remove(entry.getKey());
+entryIterator.remove();
 }
 }
 
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
index bd43685..0fecaa2 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
@@ -17,15 +17,21 @@
 
 package org.apache.kafka.streams.kstream.internals.graph;
 
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsBuilder;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.kstream.Grouped;
 import org.apache.kafka.streams.kstream.JoinWindows;
 import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.streams.kstream.Produced;
 import org.apache.kafka.streams.kstream.TimeWindows;
 import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.junit.Test;
 
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -68,6 +74,33 @@ public class StreamsGraphTest {
 }
 
 @Test
+public void shouldBeAbleToProcessNestedMultipleKeyChangingNodes() {
+final Properties properties = new Properties();
+properties.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, 
"test-application");
+properties.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
"localhost:9092");
+properties.setProperty(StreamsConfig.TOPOLOGY_OPTIMIZATION, 
StreamsConfig.OPTIMIZE);
+
+final StreamsBuilder builder = new StreamsBuilder();
+final KStream inputStream = 
builder.stream("inputTopic");
+
+final KStream changedKeyStream = 
inputStream.selectKey((k, v) -> v.substring(0, 5));
+
+// first repartition
+changedKeyStream.groupByKey(Grouped.as("count-repartition"))
+.count(Materialized.as("count-store"))
+.toStream().to("count-topic", Produced.with(Serdes.String(), 
Serdes.Long()));
+
+// sec

[kafka] branch 2.1 updated: KAFKA-8298: Fix possible concurrent modification exception (#6643)

2019-05-01 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.1 by this push:
 new d78e392  KAFKA-8298: Fix possible concurrent modification exception 
(#6643)
d78e392 is described below

commit d78e3922f7317ca8302e8f54c676f2741a99c0db
Author: Bill Bejeck 
AuthorDate: Wed May 1 12:07:45 2019 -0400

KAFKA-8298: Fix possible concurrent modification exception (#6643)

When processing multiple key-changing operations during the optimization 
phase a ConcurrentModificationException is possible.

Reviewers: Guozhang Wang 
---
 .../kstream/internals/InternalStreamsBuilder.java  |  8 --
 .../kstream/internals/graph/StreamsGraphTest.java  | 33 ++
 2 files changed, 39 insertions(+), 2 deletions(-)

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 20df084..c5e9d04 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
@@ -43,9 +43,11 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Objects;
 import java.util.PriorityQueue;
 import java.util.Properties;
@@ -305,8 +307,10 @@ public class InternalStreamsBuilder implements 
InternalNameProvider {
 @SuppressWarnings("unchecked")
 private void maybeOptimizeRepartitionOperations() {
 maybeUpdateKeyChangingRepartitionNodeMap();
+final Iterator>> entryIterator =  
keyChangingOperationsToOptimizableRepartitionNodes.entrySet().iterator();
 
-for (final Map.Entry> entry : 
keyChangingOperationsToOptimizableRepartitionNodes.entrySet()) {
+while (entryIterator.hasNext()) {
+final Map.Entry> entry = entryIterator.next();
 
 final StreamsGraphNode keyChangingNode = entry.getKey();
 
@@ -362,7 +366,7 @@ public class InternalStreamsBuilder implements 
InternalNameProvider {
 }
 
 keyChangingNode.addChild(optimizedSingleRepartition);
-
keyChangingOperationsToOptimizableRepartitionNodes.remove(entry.getKey());
+entryIterator.remove();
 }
 }
 
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
index bd43685..0fecaa2 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java
@@ -17,15 +17,21 @@
 
 package org.apache.kafka.streams.kstream.internals.graph;
 
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsBuilder;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.kstream.Grouped;
 import org.apache.kafka.streams.kstream.JoinWindows;
 import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.streams.kstream.Produced;
 import org.apache.kafka.streams.kstream.TimeWindows;
 import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.junit.Test;
 
+import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -68,6 +74,33 @@ public class StreamsGraphTest {
 }
 
 @Test
+public void shouldBeAbleToProcessNestedMultipleKeyChangingNodes() {
+final Properties properties = new Properties();
+properties.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, 
"test-application");
+properties.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
"localhost:9092");
+properties.setProperty(StreamsConfig.TOPOLOGY_OPTIMIZATION, 
StreamsConfig.OPTIMIZE);
+
+final StreamsBuilder builder = new StreamsBuilder();
+final KStream inputStream = 
builder.stream("inputTopic");
+
+final KStream changedKeyStream = 
inputStream.selectKey((k, v) -> v.substring(0, 5));
+
+// first repartition
+changedKeyStream.groupByKey(Grouped.as("count-repartition"))
+.count(Materialized.as("count-store"))
+.toStream().to("count-topic", Produced.with(Serdes.String(), 
Serdes.Long()));
+
+// sec

[kafka] branch trunk updated: KAFKA-8240: Fix NPE in Source.equals() (#6589)

2019-05-03 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 6d3ff13  KAFKA-8240: Fix NPE in Source.equals() (#6589)
6d3ff13 is described below

commit 6d3ff132b57835fc879d678e9addc5e7c3804205
Author: Matthias J. Sax 
AuthorDate: Fri May 3 16:32:34 2019 +0200

KAFKA-8240: Fix NPE in Source.equals() (#6589)

Reviewers: John Roesler , Bruno Cadonna 
, Bill Bejeck 
---
 .../internals/InternalTopologyBuilder.java |  25 +++-
 .../internals/InternalTopologyBuilderTest.java | 133 +++--
 2 files changed, 121 insertions(+), 37 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
index 792df53..2d527e5 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
@@ -282,7 +282,7 @@ public class InternalTopologyBuilder {
 
 @Override
 Source describe() {
-return new Source(name, new HashSet<>(topics), pattern);
+return new Source(name, topics.size() == 0 ? null : new 
HashSet<>(topics), pattern);
 }
 }
 
@@ -1281,6 +1281,9 @@ public class InternalTopologyBuilder {
 @Override
 public int compare(final TopologyDescription.Node node1,
final TopologyDescription.Node node2) {
+if (node1.equals(node2)) {
+return 0;
+}
 final int size1 = ((AbstractNode) node1).size;
 final int size2 = ((AbstractNode) node2).size;
 
@@ -1399,6 +1402,7 @@ public class InternalTopologyBuilder {
 int size;
 
 AbstractNode(final String name) {
+Objects.requireNonNull(name, "name cannot be null");
 this.name = name;
 this.size = 1;
 }
@@ -1435,6 +1439,13 @@ public class InternalTopologyBuilder {
   final Set topics,
   final Pattern pattern) {
 super(name);
+if (topics == null && pattern == null) {
+throw new IllegalArgumentException("Either topics or pattern 
must be not-null, but both are null.");
+}
+if (topics != null && pattern != null) {
+throw new IllegalArgumentException("Either topics or pattern 
must be null, but both are not null.");
+}
+
 this.topics = topics;
 this.topicPattern = pattern;
 }
@@ -1479,8 +1490,10 @@ public class InternalTopologyBuilder {
 final Source source = (Source) o;
 // omit successor to avoid infinite loops
 return name.equals(source.name)
-&& topics.equals(source.topics)
-&& topicPattern.equals(source.topicPattern);
+&& (topics == null && source.topics == null
+|| topics != null && topics.equals(source.topics))
+&& (topicPattern == null && source.topicPattern == null
+|| topicPattern != null && 
topicPattern.pattern().equals(source.topicPattern.pattern()));
 }
 
 @Override
@@ -1709,6 +1722,9 @@ public class InternalTopologyBuilder {
 @Override
 public int compare(final TopologyDescription.GlobalStore globalStore1,
final TopologyDescription.GlobalStore globalStore2) 
{
+if (globalStore1.equals(globalStore2)) {
+return 0;
+}
 return globalStore1.id() - globalStore2.id();
 }
 }
@@ -1719,6 +1735,9 @@ public class InternalTopologyBuilder {
 @Override
 public int compare(final TopologyDescription.Subtopology subtopology1,
final TopologyDescription.Subtopology subtopology2) 
{
+if (subtopology1.equals(subtopology2)) {
+return 0;
+}
 return subtopology1.id() - subtopology2.id();
 }
 }
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java
index 1ac7283..b86211f 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java
@@ -23,15 +23,13 @@ import org.apache.kafka.strea

[kafka] branch 2.2 updated: KAFKA-8289: Fix Session Expiration and Suppression (#6654) (#6670)

2019-05-03 Thread bbejeck
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch 2.2
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/2.2 by this push:
 new bfad7f9  KAFKA-8289: Fix Session Expiration and Suppression (#6654) 
(#6670)
bfad7f9 is described below

commit bfad7f99f7b27348245189f8cd451250e4fab3e6
Author: John Roesler 
AuthorDate: Fri May 3 15:23:59 2019 -0500

KAFKA-8289: Fix Session Expiration and Suppression (#6654) (#6670)

Fix two problems in Streams:
* Session windows expired prematurely (off-by-one error), since the window 
end is inclusive, unlike other windows
* Suppress duration for sessions incorrectly waited only the grace period, 
but session windows aren't closed until gracePeriod + sessionGap

cherry-pick of 6654 into trunk

Reviewers: Bill Bejeck 
---
 .../internals/KStreamSessionWindowAggregate.java   | 28 +++
 .../internals/graph/GraphGraceSearchUtil.java  |  2 +-
 ...KStreamSessionWindowAggregateProcessorTest.java | 96 --
 .../kstream/internals/SuppressScenarioTest.java| 41 -
 .../internals/graph/GraphGraceSearchUtilTest.java  | 10 +--
 5 files changed, 128 insertions(+), 49 deletions(-)

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java
index 2c98c10..aedf7f2 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java
@@ -133,19 +133,7 @@ public class KStreamSessionWindowAggregate 
implements KStreamAggProce
 }
 }
 
-if (mergedWindow.end() > closeTime) {
-if (!mergedWindow.equals(newSessionWindow)) {
-for (final KeyValue, Agg> session : merged) {
-store.remove(session.key);
-tupleForwarder.maybeForward(session.key, null, 
sendOldValues ? session.value : null);
-}
-}
-
-agg = aggregator.apply(key, value, agg);
-final Windowed sessionKey = new Windowed<>(key, 
mergedWindow);
-store.put(sessionKey, agg);
-tupleForwarder.maybeForward(sessionKey, agg, null);
-} else {
+if (mergedWindow.end() < closeTime) {
 LOG.debug(
 "Skipping record for expired window. " +
 "key=[{}] " +
@@ -153,7 +141,7 @@ public class KStreamSessionWindowAggregate 
implements KStreamAggProce
 "partition=[{}] " +
 "offset=[{}] " +
 "timestamp=[{}] " +
-"window=[{},{}) " +
+"window=[{},{}] " +
 "expiration=[{}] " +
 "streamTime=[{}]",
 key,
@@ -167,6 +155,18 @@ public class KStreamSessionWindowAggregate 
implements KStreamAggProce
 observedStreamTime
 );
 lateRecordDropSensor.record();
+} else {
+if (!mergedWindow.equals(newSessionWindow)) {
+for (final KeyValue, Agg> session : merged) {
+store.remove(session.key);
+tupleForwarder.maybeForward(session.key, null, 
sendOldValues ? session.value : null);
+}
+}
+
+agg = aggregator.apply(key, value, agg);
+final Windowed sessionKey = new Windowed<>(key, 
mergedWindow);
+store.put(sessionKey, agg);
+tupleForwarder.maybeForward(sessionKey, agg, null);
 }
 }
 }
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtil.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtil.java
index 306ddf5..2fb28dd 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtil.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtil.java
@@ -78,7 +78,7 @@ public final class GraphGraceSearchUtil {
 } else if (processorSupplier instanceof 
KStreamSessionWindowAggregate) {
 final KStreamSessionWindowAggregate 
kStreamSessionWindowAggregate = (KStreamSessionWindowAggregate) 
processorSupplier;
 final SessionWindows windows = 
kStreamSessionWindowAggregate.windows();
-return 

  1   2   3   4   5   6   7   >