You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by ja...@apache.org on 2018/10/31 22:01:54 UTC

[1/4] samza git commit: Fix to make Samza SQL applications work after the Runner refactoring

Repository: samza
Updated Branches:
  refs/heads/1.0.0 1d5fc720b -> ed8d1da8b


Fix to make Samza SQL applications work after the Runner refactoring

With recent change in Samza, Constructor signature for ApplicationRunner has changed. But the SamzaSQLApplicationRunner was not updated with the new signature. This is to fix the signature of the constructor for SamzaSQLApplicationRunner with the updated signature.

Author: Srinivasulu Punuru <sp...@linkedin.com>

Reviewers: Aditya Toomula <at...@linkedin.com>

Closes #784 from srinipunuru/sql-app-fix.1


Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/a664fb51
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/a664fb51
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/a664fb51

Branch: refs/heads/1.0.0
Commit: a664fb51877394bc0e3aea428bd4699a70d9b059
Parents: 1d5fc72
Author: Srinivasulu Punuru <sp...@linkedin.com>
Authored: Tue Oct 30 13:33:33 2018 -0700
Committer: Jagadish <jv...@linkedin.com>
Committed: Wed Oct 31 14:29:51 2018 -0700

----------------------------------------------------------------------
 .../sql/runner/SamzaSqlApplicationRunner.java   | 55 ++++++++++++--------
 1 file changed, 33 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/a664fb51/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java
index cad032f..9d361fb 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java
@@ -1,21 +1,21 @@
 /*
-* 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.
-*/
+ * 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.samza.sql.runner;
 
@@ -26,6 +26,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import org.apache.commons.lang3.Validate;
+import org.apache.samza.application.SamzaApplication;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.job.ApplicationStatus;
@@ -56,9 +57,20 @@ public class SamzaSqlApplicationRunner implements ApplicationRunner {
   public static final String RUNNER_CONFIG = "app.runner.class";
   public static final String CFG_FMT_SAMZA_STREAM_SYSTEM = "streams.%s.samza.system";
 
+  /**
+   * NOTE: This constructor is called from {@link ApplicationRunners} through reflection.
+   * Please refrain from updating the signature or removing this constructor unless the caller has changed the interface.
+   */
+  public SamzaSqlApplicationRunner(SamzaApplication app, Config config) {
+    this(app, false, config);
+  }
+
   public SamzaSqlApplicationRunner(Boolean localRunner, Config config) {
-    this.runner = ApplicationRunners.getApplicationRunner(new SamzaSqlApplication(),
-        computeSamzaConfigs(localRunner, config));
+    this(new SamzaSqlApplication(), localRunner, config);
+  }
+
+  private SamzaSqlApplicationRunner(SamzaApplication app, Boolean localRunner, Config config) {
+    this.runner = ApplicationRunners.getApplicationRunner(app, computeSamzaConfigs(localRunner, config));
   }
 
   public static Config computeSamzaConfigs(Boolean localRunner, Config config) {
@@ -74,8 +86,8 @@ public class SamzaSqlApplicationRunner implements ApplicationRunner {
     Set<String> inputSystemStreams = new HashSet<>();
     Set<String> outputSystemStreams = new HashSet<>();
 
-    SamzaSqlApplicationConfig.populateSystemStreamsAndGetRelRoots(dslStmts, config,
-        inputSystemStreams, outputSystemStreams);
+    SamzaSqlApplicationConfig.populateSystemStreamsAndGetRelRoots(dslStmts, config, inputSystemStreams,
+        outputSystemStreams);
 
     SqlIOResolver ioResolver = SamzaSqlApplicationConfig.createIOResolver(config);
 
@@ -136,5 +148,4 @@ public class SamzaSqlApplicationRunner implements ApplicationRunner {
   public boolean waitForFinish(Duration timeout) {
     return runner.waitForFinish(timeout);
   }
-
 }


[3/4] samza git commit: SAMZA-1971: Fix NPE in partition key computation for InMemorySystemProducer

Posted by ja...@apache.org.
SAMZA-1971: Fix NPE in partition key computation for InMemorySystemProducer

Author: bharathkk <co...@gmail.com>

Reviewers: Jagadish<ja...@apache.org>

Closes #786 from bharathkk/fix-inmemory-partitionkey-npe


Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/5ddbd30b
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/5ddbd30b
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/5ddbd30b

Branch: refs/heads/1.0.0
Commit: 5ddbd30b09806e1df776a732b8d9c43c4543c278
Parents: 7a00020
Author: bharathkk <co...@gmail.com>
Authored: Wed Oct 31 12:41:20 2018 -0700
Committer: Jagadish <jv...@linkedin.com>
Committed: Wed Oct 31 14:30:28 2018 -0700

----------------------------------------------------------------------
 .../system/inmemory/InMemorySystemProducer.java | 22 ++++++++++-----
 .../system/inmemory/TestInMemorySystem.java     | 28 ++++++++++++++++++++
 2 files changed, 44 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/5ddbd30b/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemProducer.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemProducer.java b/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemProducer.java
index cd5e649..872488d 100644
--- a/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemProducer.java
+++ b/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemProducer.java
@@ -19,7 +19,7 @@
 
 package org.apache.samza.system.inmemory;
 
-import java.util.Optional;
+import com.google.common.base.Preconditions;
 import org.apache.samza.Partition;
 import org.apache.samza.system.OutgoingMessageEnvelope;
 import org.apache.samza.system.SystemProducer;
@@ -75,11 +75,21 @@ public class InMemorySystemProducer implements SystemProducer {
     Object key = envelope.getKey();
     Object message = envelope.getMessage();
 
-    // use the hashcode from partition key in the outgoing message envelope or default to message hashcode
-    int hashCode = Optional.ofNullable(envelope.getPartitionKey())
-        .map(Object::hashCode)
-        .orElse(message.hashCode());
-    int partition = Math.abs(hashCode) % memoryManager.getPartitionCountForSystemStream(envelope.getSystemStream());
+    Object partitionKey;
+    // We use the partition key from message if available, if not fallback to message key or use message as partition
+    // key as the final resort.
+    if (envelope.getPartitionKey() != null) {
+      partitionKey = envelope.getPartitionKey();
+    } else if (key != null) {
+      partitionKey = key;
+    } else {
+      partitionKey = message;
+    }
+
+    Preconditions.checkNotNull(partitionKey, "Failed to compute partition key for the message: " + envelope);
+
+    int partition =
+        Math.abs(partitionKey.hashCode()) % memoryManager.getPartitionCountForSystemStream(envelope.getSystemStream());
 
     SystemStreamPartition ssp = new SystemStreamPartition(envelope.getSystemStream(), new Partition(partition));
     memoryManager.put(ssp, key, message);

http://git-wip-us.apache.org/repos/asf/samza/blob/5ddbd30b/samza-core/src/test/java/org/apache/samza/system/inmemory/TestInMemorySystem.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/system/inmemory/TestInMemorySystem.java b/samza-core/src/test/java/org/apache/samza/system/inmemory/TestInMemorySystem.java
index 7d5dfd0..0a2e221 100644
--- a/samza-core/src/test/java/org/apache/samza/system/inmemory/TestInMemorySystem.java
+++ b/samza-core/src/test/java/org/apache/samza/system/inmemory/TestInMemorySystem.java
@@ -142,6 +142,34 @@ public class TestInMemorySystem {
     assertTrue(results.get(0).isEndOfStream());
   }
 
+  @Test
+  public void testNullMessageWithValidMessageKey() {
+    final String messageKey = "validKey";
+    SystemProducer systemProducer = systemFactory.getProducer(SYSTEM_NAME, config, mockRegistry);
+    systemProducer.send(SOURCE, new OutgoingMessageEnvelope(SYSTEM_STREAM, messageKey, null));
+
+    SystemConsumer consumer = systemFactory.getConsumer(SYSTEM_NAME, config, mockRegistry);
+
+    Set<SystemStreamPartition> sspsToPoll = IntStream.range(0, PARTITION_COUNT)
+        .mapToObj(partition -> new SystemStreamPartition(SYSTEM_STREAM, new Partition(partition)))
+        .collect(Collectors.toSet());
+
+    // register the consumer for ssps
+    for (SystemStreamPartition ssp : sspsToPoll) {
+      consumer.register(ssp, "0");
+    }
+
+    List<IncomingMessageEnvelope> results = consumeRawMessages(consumer, sspsToPoll);
+    assertEquals(1, results.size());
+    assertEquals(results.get(0).getKey(), messageKey);
+    assertNull(results.get(0).getMessage());
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testNullMessageWithNullKey() {
+    SystemProducer systemProducer = systemFactory.getProducer(SYSTEM_NAME, config, mockRegistry);
+    systemProducer.send(SOURCE, new OutgoingMessageEnvelope(SYSTEM_STREAM, null));
+  }
 
   private <T> List<T> consumeMessages(Set<SystemStreamPartition> sspsToPoll) {
     SystemConsumer systemConsumer = systemFactory.getConsumer(SYSTEM_NAME, config, mockRegistry);


[2/4] samza git commit: Close iterators to time-series store on deletes

Posted by ja...@apache.org.
Close iterators to time-series store on deletes

Author: Jagadish <jv...@linkedin.com>

Reviewers: Jagadish<ja...@apache.org>

Closes #787 from vjagadish1989/website-reorg29


Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/7a000206
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/7a000206
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/7a000206

Branch: refs/heads/1.0.0
Commit: 7a000206f52617dec92b67efb91b59a2ec12c795
Parents: a664fb5
Author: Jagadish <jv...@linkedin.com>
Authored: Tue Oct 30 18:48:40 2018 -0700
Committer: Jagadish <jv...@linkedin.com>
Committed: Wed Oct 31 14:30:14 2018 -0700

----------------------------------------------------------------------
 .../org/apache/samza/operators/impl/WindowOperatorImpl.java | 4 ++--
 .../samza/operators/impl/store/TimeSeriesStoreImpl.java     | 9 ++++++---
 2 files changed, 8 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/7a000206/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
index c09c5f8..0241d9e 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/WindowOperatorImpl.java
@@ -379,15 +379,15 @@ public class WindowOperatorImpl<M, K> extends OperatorImpl<M, WindowPane<K, Obje
    * @return a list of all elements returned by the iterator
    */
   static <V>  List<V> toList(ClosableIterator<V> iterator) {
+    Preconditions.checkNotNull(iterator);
+
     List<V> values = new ArrayList<>();
     try {
       while (iterator.hasNext()) {
         values.add(iterator.next());
       }
     } finally {
-      if (iterator != null) {
         iterator.close();
-      }
     }
     return Collections.unmodifiableList(values);
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/7a000206/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java
index 10a5967..b8cd82f 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/impl/store/TimeSeriesStoreImpl.java
@@ -154,10 +154,13 @@ public class TimeSeriesStoreImpl<K, V> implements TimeSeriesStore<K, V> {
     List<TimeSeriesKey<K>> keysToDelete = new LinkedList<>();
 
     KeyValueIterator<TimeSeriesKey<K>, V> range = kvStore.range(fromKey, toKey);
-    while (range.hasNext()) {
-      keysToDelete.add(range.next().getKey());
+    try {
+      while (range.hasNext()) {
+        keysToDelete.add(range.next().getKey());
+      }
+    } finally {
+      range.close();
     }
-
     kvStore.deleteAll(keysToDelete);
   }
 


[4/4] samza git commit: SAMZA-1970: Support for physical names in InMemorySystem

Posted by ja...@apache.org.
SAMZA-1970: Support for physical names in InMemorySystem

if super is not there, java compiles this to this.withPhysicalName which results in StackOverflows

Author: Sanil15 <sa...@gmail.com>

Reviewers: Jagadish<ja...@apache.org>

Closes #788 from Sanil15/SAMZA-1970-edit


Project: http://git-wip-us.apache.org/repos/asf/samza/repo
Commit: http://git-wip-us.apache.org/repos/asf/samza/commit/ed8d1da8
Tree: http://git-wip-us.apache.org/repos/asf/samza/tree/ed8d1da8
Diff: http://git-wip-us.apache.org/repos/asf/samza/diff/ed8d1da8

Branch: refs/heads/1.0.0
Commit: ed8d1da8bdf3dfd9fc2ce49907561105d6429a08
Parents: 5ddbd30
Author: Sanil15 <sa...@gmail.com>
Authored: Wed Oct 31 12:41:40 2018 -0700
Committer: Jagadish <jv...@linkedin.com>
Committed: Wed Oct 31 14:30:39 2018 -0700

----------------------------------------------------------------------
 .../test/framework/system/descriptors/InMemoryInputDescriptor.java | 2 +-
 .../framework/system/descriptors/InMemoryOutputDescriptor.java     | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/ed8d1da8/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryInputDescriptor.java b/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryInputDescriptor.java
index f2214de..477c35d 100644
--- a/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryInputDescriptor.java
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryInputDescriptor.java
@@ -41,7 +41,7 @@ public class InMemoryInputDescriptor<StreamMessageType>
   }
 
   public InMemoryInputDescriptor withPhysicalName(String physicalName) {
-    withPhysicalName(physicalName);
+    super.withPhysicalName(physicalName);
     return this;
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/ed8d1da8/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryOutputDescriptor.java b/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryOutputDescriptor.java
index 7c38aa6..a86e5a1 100644
--- a/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryOutputDescriptor.java
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/system/descriptors/InMemoryOutputDescriptor.java
@@ -45,7 +45,7 @@ public class InMemoryOutputDescriptor<StreamMessageType>
   }
 
   public InMemoryOutputDescriptor withPhysicalName(String physicalName) {
-    withPhysicalName(physicalName);
+    super.withPhysicalName(physicalName);
     return this;
   }
 }