From 8cad71d7af9d8a477d65e12de1007db2106c5b27 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Thu, 21 Nov 2019 20:30:45 -0800 Subject: [PATCH 01/49] SAMZA-2397: Updating gauge-val function on newGauge on same metric name --- .../scala/org/apache/samza/metrics/MetricsRegistryMap.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala index 40ffee2bfa..3df30d06a3 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala @@ -51,7 +51,10 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with def newGauge[T](group: String, gauge: Gauge[T]) = { debug("Adding new gauge %s %s %s." format (group, gauge.getName, gauge)) - putAndGetGroup(group).putIfAbsent(gauge.getName, gauge) + if (putAndGetGroup(group).containsKey(gauge.getName)) { + info("Updating existing gauge %s %s %s" format (group, gauge.getName, gauge)) + } + putAndGetGroup(group).put(gauge.getName, gauge) val realGauge = metrics.get(group).get(gauge.getName).asInstanceOf[Gauge[T]] listeners.foreach(_.onGauge(group, realGauge)) realGauge From f64d6cfe529c9c9a7f33662a8241e0849791b04d Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Thu, 21 Nov 2019 21:58:23 -0800 Subject: [PATCH 02/49] Adding unit test for metrics --- .../storage/kv/TestRocksDbKeyValueStore.scala | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala b/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala index ca9c023ad8..356ca500df 100644 --- a/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala +++ b/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala @@ -210,4 +210,27 @@ class TestRocksDbKeyValueStore rocksDB.close() } + + @Test + def testRocksDBMetricsWithBulkLoadRWRecreate(): Unit = { + val registry = new MetricsRegistryMap("registrymap") + val metrics = new KeyValueStoreMetrics("dbstore", registry) + + // Sample metric values for estimate-num-keys metrics + val bulkloadStoreMetricValue = "100" + val readWriteStoreMetricValue = "10" + + // Metric during bulk-load/bootstrap + metrics.newGauge("estimate-num-keys", () => bulkloadStoreMetricValue) + + assert(registry.getGroup("org.apache.samza.storage.kv.KeyValueStoreMetrics"). + get("dbstore-estimate-num-keys").asInstanceOf[Gauge[String]].getValue.eq("100")) + + // Bulk-load complete, new store in read-write mode + metrics.newGauge("estimate-num-keys", () => readWriteStoreMetricValue) + + assert(registry.getGroup("org.apache.samza.storage.kv.KeyValueStoreMetrics"). + get("dbstore-estimate-num-keys").asInstanceOf[Gauge[String]].getValue.eq("10")) + } + } From 74988558f46a9765aae3da560ea3be9a5215349a Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Fri, 22 Nov 2019 10:17:10 -0800 Subject: [PATCH 03/49] Adding separate method on MetricsHelper to get new gauge with overwrite --- .../apache/samza/metrics/MetricsRegistry.java | 11 +++++++++++ .../apache/samza/util/NoOpMetricsRegistry.java | 5 +++++ .../system/eventhub/TestMetricsRegistry.java | 7 +++++++ .../org/apache/samza/metrics/MetricGroup.java | 9 +++++++++ .../apache/samza/metrics/MetricsHelper.scala | 6 ++++++ .../samza/metrics/MetricsRegistryMap.scala | 18 +++++++++++++++--- .../storage/kv/RocksDbKeyValueStore.scala | 2 +- .../storage/kv/TestRocksDbKeyValueStore.scala | 6 +++--- .../sql/util/TestMetricsRegistryImpl.java | 7 +++++++ 9 files changed, 64 insertions(+), 7 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java b/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java index 5a00d01868..f5a4a438c0 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java @@ -64,6 +64,17 @@ public interface MetricsRegistry { */ Gauge newGauge(String group, Gauge value); + /** + * Register an existing {@link org.apache.samza.metrics.Gauge} + * @param group Group for this Gauge + * @param value Initial value for the Gauge + * @param Type the Gauge will be wrapping + * @param overrideExistingGauge Overwrite any existing gauges present for the same group and gauge + * @return Gauge was registered + */ + Gauge newGauge(String group, Gauge value, Boolean overrideExistingGauge); + + /** * Create and Register a new {@link org.apache.samza.metrics.Timer} * @param group Group for this Timer diff --git a/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java b/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java index 068fe72bb1..e2519f171b 100644 --- a/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java +++ b/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java @@ -50,6 +50,11 @@ public Gauge newGauge(String group, Gauge gauge) { return gauge; } + @Override + public Gauge newGauge(String group, Gauge gauge, Boolean overrideExistingGauge) { + return gauge; + } + @Override public Timer newTimer(String group, String name) { return new Timer(name); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java index 48f6edda29..534b70563d 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java @@ -77,6 +77,13 @@ public Gauge newGauge(String group, Gauge value) { return value; } + @Override + public Gauge newGauge(String group, Gauge gauge, Boolean overrideExistingGauge) { + gauges.putIfAbsent(group, new ArrayList<>()); + gauges.get(group).add(gauge); + return gauge; + } + @Override public Timer newTimer(String group, String name) { return null; diff --git a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java index 53526d86c4..870f1af150 100644 --- a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java +++ b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java @@ -62,6 +62,15 @@ public T getValue() { }); } + public Gauge newGauge(String name, final ValueFunction valueFunc, Boolean overwrite) { + return registry.newGauge(groupName, new Gauge((prefix + name).toLowerCase(), valueFunc.getValue()) { + @Override + public T getValue() { + return valueFunc.getValue(); + } + }, overwrite); + } + public Timer newTimer(String name) { return registry.newTimer(groupName, (prefix + name).toLowerCase()); } diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala index 6e1fb8cf6c..64cb240ca6 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala @@ -51,6 +51,12 @@ trait MetricsHelper { }) } + def newGauge[T](name: String, value: () => T, overwrite : Boolean) = { + metricGroup.newGauge(name, new ValueFunction[T] { + override def getValue = value() + }, overwrite) + } + /** * Returns a prefix for metric names. */ diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala index 3df30d06a3..9f7e556386 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala @@ -19,6 +19,8 @@ package org.apache.samza.metrics +import java.lang + import org.apache.samza.util.Logging import java.util.concurrent.ConcurrentHashMap @@ -51,10 +53,20 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with def newGauge[T](group: String, gauge: Gauge[T]) = { debug("Adding new gauge %s %s %s." format (group, gauge.getName, gauge)) - if (putAndGetGroup(group).containsKey(gauge.getName)) { - info("Updating existing gauge %s %s %s" format (group, gauge.getName, gauge)) + putAndGetGroup(group).putIfAbsent(gauge.getName, gauge) + val realGauge = metrics.get(group).get(gauge.getName).asInstanceOf[Gauge[T]] + listeners.foreach(_.onGauge(group, realGauge)) + realGauge + } + + def newGauge[T](group: String, gauge: Gauge[T], overrideExistingGauge: lang.Boolean) = { + if (overrideExistingGauge) { + debug("Updating existing gauge %s %s %s" format (group, gauge.getName, gauge)) + putAndGetGroup(group).put(gauge.getName, gauge) + } else { + debug("Adding new gauge %s %s %s." format (group, gauge.getName, gauge)) + putAndGetGroup(group).putIfAbsent(gauge.getName, gauge) } - putAndGetGroup(group).put(gauge.getName, gauge) val realGauge = metrics.get(group).get(gauge.getName).asInstanceOf[Gauge[T]] listeners.foreach(_.onGauge(group, realGauge)) realGauge diff --git a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala index 300177af6a..33724427c2 100644 --- a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala +++ b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala @@ -98,7 +98,7 @@ object RocksDbKeyValueStore extends Logging { rocksDb.getProperty(property) } else { "0" - } + }, true // overrite-gauge = true, to accomodate for store-close and re-open (after bulk-load) )) rocksDb diff --git a/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala b/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala index 356ca500df..f06797558c 100644 --- a/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala +++ b/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala @@ -25,7 +25,7 @@ import java.util import org.apache.samza.SamzaException import org.apache.samza.config.MapConfig -import org.apache.samza.metrics.{Gauge, MetricsRegistryMap} +import org.apache.samza.metrics.{Counter, Gauge, MetricsRegistryMap, MetricsVisitor, Timer} import org.apache.samza.util.ExponentialSleepStrategy import org.junit.{Assert, Test} import org.rocksdb.{FlushOptions, Options, RocksDB, RocksIterator} @@ -221,13 +221,13 @@ class TestRocksDbKeyValueStore val readWriteStoreMetricValue = "10" // Metric during bulk-load/bootstrap - metrics.newGauge("estimate-num-keys", () => bulkloadStoreMetricValue) + metrics.newGauge("estimate-num-keys", () => bulkloadStoreMetricValue, true) assert(registry.getGroup("org.apache.samza.storage.kv.KeyValueStoreMetrics"). get("dbstore-estimate-num-keys").asInstanceOf[Gauge[String]].getValue.eq("100")) // Bulk-load complete, new store in read-write mode - metrics.newGauge("estimate-num-keys", () => readWriteStoreMetricValue) + metrics.newGauge("estimate-num-keys", () => readWriteStoreMetricValue.toString, true) assert(registry.getGroup("org.apache.samza.storage.kv.KeyValueStoreMetrics"). get("dbstore-estimate-num-keys").asInstanceOf[Gauge[String]].getValue.eq("10")) diff --git a/samza-sql/src/test/java/org/apache/samza/sql/util/TestMetricsRegistryImpl.java b/samza-sql/src/test/java/org/apache/samza/sql/util/TestMetricsRegistryImpl.java index 9a4b8c9eb1..b594a527ce 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/util/TestMetricsRegistryImpl.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/util/TestMetricsRegistryImpl.java @@ -97,6 +97,13 @@ public Gauge newGauge(String group, Gauge gauge) { return gauge; } + @Override + public Gauge newGauge(String group, Gauge gauge, Boolean overrideExistingGauge) { + gauges.putIfAbsent(group, new ArrayList<>()); + gauges.get(group).add(gauge); + return gauge; + } + /** * retrieves the Map of Gauges * @return gauges From 245eefd7708954800a66da935b4232586feb472d Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Fri, 22 Nov 2019 10:17:10 -0800 Subject: [PATCH 04/49] Adding separate method on MetricsHelper to get new gauge with overwrite Updating method in MetricsRegistryMap --- .../scala/org/apache/samza/metrics/MetricsRegistryMap.scala | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala index 9f7e556386..5a291fe7a4 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala @@ -52,11 +52,7 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with } def newGauge[T](group: String, gauge: Gauge[T]) = { - debug("Adding new gauge %s %s %s." format (group, gauge.getName, gauge)) - putAndGetGroup(group).putIfAbsent(gauge.getName, gauge) - val realGauge = metrics.get(group).get(gauge.getName).asInstanceOf[Gauge[T]] - listeners.foreach(_.onGauge(group, realGauge)) - realGauge + newGauge(group, gauge, false) } def newGauge[T](group: String, gauge: Gauge[T], overrideExistingGauge: lang.Boolean) = { From 4764adef03480fe6a7f62aab76a3b13754fab781 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Mon, 2 Dec 2019 14:03:12 -0800 Subject: [PATCH 05/49] Empty commit to trigger build From 959b3d8ba1f198939e76c5eef1eb4cffe60b335e Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Thu, 12 Dec 2019 12:24:44 -0800 Subject: [PATCH 06/49] Addressing review comments --- .../java/org/apache/samza/metrics/MetricGroup.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java index 870f1af150..dec56c4356 100644 --- a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java +++ b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java @@ -54,21 +54,17 @@ public Gauge newGauge(String name, T value) { * it from another thread. */ public Gauge newGauge(String name, final ValueFunction valueFunc) { - return registry.newGauge(groupName, new Gauge((prefix + name).toLowerCase(), valueFunc.getValue()) { - @Override - public T getValue() { - return valueFunc.getValue(); - } - }); + return this.newGauge(name, valueFunc, false); } - public Gauge newGauge(String name, final ValueFunction valueFunc, Boolean overwrite) { + // TODO SAMZA-2417 make overwriting the default behavior + public Gauge newGauge(String name, final ValueFunction valueFunc, Boolean overrideExistingGauge) { return registry.newGauge(groupName, new Gauge((prefix + name).toLowerCase(), valueFunc.getValue()) { @Override public T getValue() { return valueFunc.getValue(); } - }, overwrite); + }, overrideExistingGauge); } public Timer newTimer(String name) { From 16256fb51e27b9689319b041093c601d17593225 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Thu, 12 Dec 2019 16:02:16 -0800 Subject: [PATCH 07/49] Minor naming changes --- .../main/scala/org/apache/samza/metrics/MetricsHelper.scala | 4 ++-- .../org/apache/samza/storage/kv/RocksDbKeyValueStore.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala index 64cb240ca6..cf19c3fc64 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala @@ -51,10 +51,10 @@ trait MetricsHelper { }) } - def newGauge[T](name: String, value: () => T, overwrite : Boolean) = { + def newGauge[T](name: String, value: () => T, overrideExistingGauge : Boolean) = { metricGroup.newGauge(name, new ValueFunction[T] { override def getValue = value() - }, overwrite) + }, overrideExistingGauge) } /** diff --git a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala index 33724427c2..40fd0258ce 100644 --- a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala +++ b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala @@ -98,7 +98,7 @@ object RocksDbKeyValueStore extends Logging { rocksDb.getProperty(property) } else { "0" - }, true // overrite-gauge = true, to accomodate for store-close and re-open (after bulk-load) + }, true // overwrite-gauge = true, to accommodate for store-close and re-open (after bulk-load) )) rocksDb From 7e558bfc2db72837fef3e161e918b8d5fb2bbcaf Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Fri, 17 Jan 2020 21:30:15 -0800 Subject: [PATCH 08/49] Trigger build From dfb4ceefbdacd3b2a76eadc3548b0c66eeb7148c Mon Sep 17 00:00:00 2001 From: Cameron Lee Date: Tue, 26 Nov 2019 11:13:28 -0800 Subject: [PATCH 09/49] SAMZA-2333: [AM isolation] Use cytodynamics classloader to launch job coordinator (#1173) --- build.gradle | 2 + gradle/dependency-versions.gradle | 3 +- .../classloader/DependencyIsolationUtils.java | 8 + .../IsolatingClassLoaderFactory.java | 327 ++++++++++++++++++ .../ClusterBasedJobCoordinator.java | 77 ++++- .../TestIsolatingClassLoaderFactory.java | 96 +++++ .../TestClusterBasedJobCoordinator.java | 40 ++- .../classloader/classpath/placeholder-jar.jar | 1 + .../classpath/placeholder-txt.json | 1 + .../classloader/classpath/placeholder-war.war | 1 + .../samza-framework-api-classes.txt | 6 + samza-shell/src/main/bash/run-class.sh | 39 ++- 12 files changed, 589 insertions(+), 12 deletions(-) create mode 100644 samza-core/src/main/java/org/apache/samza/classloader/IsolatingClassLoaderFactory.java create mode 100644 samza-core/src/test/java/org/apache/samza/classloader/TestIsolatingClassLoaderFactory.java create mode 100644 samza-core/src/test/resources/classloader/classpath/placeholder-jar.jar create mode 100644 samza-core/src/test/resources/classloader/classpath/placeholder-txt.json create mode 100644 samza-core/src/test/resources/classloader/classpath/placeholder-war.war create mode 100644 samza-core/src/test/resources/classloader/samza-framework-api-classes.txt diff --git a/build.gradle b/build.gradle index c4ca3491c6..dbf777ea7a 100644 --- a/build.gradle +++ b/build.gradle @@ -89,6 +89,7 @@ rat { 'samza-test/state/**', 'README.md', 'RELEASE.md', + 'samza-core/src/test/resources/classloader/samza-framework-api-classes.txt', 'samza-test/src/main/resources/**', 'samza-hdfs/src/main/resources/**', 'samza-hdfs/src/test/resources/**', @@ -190,6 +191,7 @@ project(":samza-core_$scalaSuffix") { compile "org.scala-lang:scala-library:$scalaVersion" compile "org.slf4j:slf4j-api:$slf4jVersion" compile "net.jodah:failsafe:$failsafeVersion" + compile "com.linkedin.cytodynamics:cytodynamics-nucleus:$cytodynamicsVersion" testCompile project(":samza-api").sourceSets.test.output testCompile "junit:junit:$junitVersion" testCompile "org.mockito:mockito-core:$mockitoVersion" diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index f70e879d37..fcf472e97d 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -24,8 +24,9 @@ commonsCodecVersion = "1.9" commonsCollectionVersion = "3.2.1" commonsHttpClientVersion = "3.1" - commonsLang3Version = "3.4" commonsIoVersion = "2.6" + commonsLang3Version = "3.4" + cytodynamicsVersion = "0.2.0" elasticsearchVersion = "2.2.0" gsonVersion = "2.8.5" guavaVersion = "23.0" diff --git a/samza-core/src/main/java/org/apache/samza/classloader/DependencyIsolationUtils.java b/samza-core/src/main/java/org/apache/samza/classloader/DependencyIsolationUtils.java index 0d44659c5d..8f933eac62 100644 --- a/samza-core/src/main/java/org/apache/samza/classloader/DependencyIsolationUtils.java +++ b/samza-core/src/main/java/org/apache/samza/classloader/DependencyIsolationUtils.java @@ -25,16 +25,24 @@ public class DependencyIsolationUtils { * TODO make this configurable or taken from an environment variable */ public static final String FRAMEWORK_API_DIRECTORY = "__samzaFrameworkApi"; + /** * Directory inside the home directory of the cluster-based job coordinator in which the framework infrastructure * artifacts are placed, for usage in dependency isolation for the cluster-based job coordinator. * TODO make this configurable or taken from an environment variable */ public static final String FRAMEWORK_INFRASTRUCTURE_DIRECTORY = "__samzaFrameworkInfrastructure"; + /** * Directory inside the home directory of the cluster-based job coordinator in which the application artifacts are * placed, for usage in dependency isolation for the cluster-based job coordinator. * TODO make this configurable or taken from an environment variable */ public static final String APPLICATION_DIRECTORY = "__package"; + + /** + * Name of the file which contains the class names (or globs) which should be loaded from the framework API + * classloader. + */ + public static final String FRAMEWORK_API_CLASS_LIST_FILE_NAME = "samza-framework-api-classes.txt"; } diff --git a/samza-core/src/main/java/org/apache/samza/classloader/IsolatingClassLoaderFactory.java b/samza-core/src/main/java/org/apache/samza/classloader/IsolatingClassLoaderFactory.java new file mode 100644 index 0000000000..47d1ea0734 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/classloader/IsolatingClassLoaderFactory.java @@ -0,0 +1,327 @@ +/* + * 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.classloader; + +import com.linkedin.cytodynamics.matcher.BootstrapClassPredicate; +import com.linkedin.cytodynamics.matcher.GlobMatcher; +import com.linkedin.cytodynamics.nucleus.DelegateRelationship; +import com.linkedin.cytodynamics.nucleus.DelegateRelationshipBuilder; +import com.linkedin.cytodynamics.nucleus.IsolationLevel; +import com.linkedin.cytodynamics.nucleus.LoaderBuilder; +import com.linkedin.cytodynamics.nucleus.OriginRestriction; +import java.io.File; +import java.io.IOException; +import java.net.MalformedURLException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.lang3.StringUtils; +import org.apache.samza.SamzaException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Use this to build a classloader for running Samza which isolates the Samza framework code/dependencies from the + * application code/dependencies. + */ +public class IsolatingClassLoaderFactory { + private static final Logger LOG = LoggerFactory.getLogger(IsolatingClassLoaderFactory.class); + + private static final String LIB_DIRECTORY = "lib"; + + /** + * Build a classloader which will isolate Samza framework code from application code. Samza framework classes and + * application-specific classes will be loaded using a different classloaders. This will enable dependencies of each + * category of classes to also be loaded separately, so that runtime dependency conflicts do not happen. + * Each call to this method will build a different instance of a classloader. + * + * Samza framework API classes need to be specified in a file called + * {@link DependencyIsolationUtils#FRAMEWORK_API_CLASS_LIST_FILE_NAME} which is in the lib directory which is in the + * API package. The file needs to be generated when building the framework API package. This class will not generate + * the file. + * + * Implementation notes: + * + * The cytodynamics isolating classloader is used for this. It provides more control than the built-in + * {@link URLClassLoader}. Cytodynamics provides the ability to compose multiple classloaders together and have more + * granular delegation strategies between the classloaders. + * + * In order to share objects between classes loaded by different classloaders, the classes for the shared objects must + * be loaded by a common classloader. Those common classes will be loaded through a common API classloader. The + * cytodynamics classloader can be set up to only use the common API classloader for an explicit set of classes. The + * {@link DependencyIsolationUtils#FRAMEWORK_API_CLASS_LIST_FILE_NAME} file should include the framework API classes. + * Also, bootstrap classes (e.g. java.lang.String) need to be loaded by a common classloader, since objects of those + * types need to be shared across different framework and application. There are also some static bootstrap classes + * which should be shared (e.g. java.lang.System). Bootstrap classes will be loaded through a common classloader by + * default. + * + * These are the classloaders which are used to make up the final classloader. + *
    + *
  • bootstrap classloader: Built-in Java classes (e.g. java.lang.String)
  • + *
  • API classloader: Common Samza framework API classes
  • + *
  • infrastructure classloader: Core Samza framework classes and plugins that are included in the framework
  • + *
  • + * application classloader: Application code and plugins that are needed in the app but are not included in the + * framework + *
  • + *
+ * + * This is the delegation structure for the classloaders: + *
+   *   (bootstrap               (API                  (application
+   *   classloader) <---- classloader) <------- classloader)
+   *                             ^                      ^
+   *                             |                     /
+   *                             |                    /
+   *                             |                   /
+   *                             |                  /
+   *                         (infrastructure classloader)
+   * 
+ * The cytodynamics classloader allows control over when the delegation should happen. + *
    + *
  1. API classloader delegates to the bootstrap classloader if the bootstrap classloader has the class.
  2. + *
  3. + * Infrastructure classloader only delegates to the API classloader for the common classes specified by + * {@link DependencyIsolationUtils#FRAMEWORK_API_CLASS_LIST_FILE_NAME}. + *
  4. + *
  5. + * Infrastructure classloader delegates to the application classloader when a class can't be found in the + * infrastructure classloader. + *
  6. + *
  7. + * Application classloader only delegates to the API classloader for the common classes specified by + * {@link DependencyIsolationUtils#FRAMEWORK_API_CLASS_LIST_FILE_NAME}. + *
  8. + *
+ */ + public ClassLoader buildClassLoader() { + // start at the user.dir to find the resources for the classpaths + String baseDirectoryPath = System.getProperty("user.dir"); + File apiLibDirectory = libDirectory(new File(baseDirectoryPath, DependencyIsolationUtils.FRAMEWORK_API_DIRECTORY)); + LOG.info("Using API lib directory: {}", apiLibDirectory); + File infrastructureLibDirectory = + libDirectory(new File(baseDirectoryPath, DependencyIsolationUtils.FRAMEWORK_INFRASTRUCTURE_DIRECTORY)); + LOG.info("Using infrastructure lib directory: {}", infrastructureLibDirectory); + File applicationLibDirectory = + libDirectory(new File(baseDirectoryPath, DependencyIsolationUtils.APPLICATION_DIRECTORY)); + LOG.info("Using application lib directory: {}", applicationLibDirectory); + + ClassLoader apiClassLoader = buildApiClassLoader(apiLibDirectory); + ClassLoader applicationClassLoader = + buildApplicationClassLoader(applicationLibDirectory, apiLibDirectory, apiClassLoader); + + // the classloader to return is the one with the infrastructure classpath + return buildInfrastructureClassLoader(infrastructureLibDirectory, apiLibDirectory, apiClassLoader, + applicationClassLoader); + } + + /** + * Build the {@link ClassLoader} which can load framework API classes. + * + * This sets up the link between the bootstrap classloader and the API classloader (see {@link #buildClassLoader()}. + */ + private static ClassLoader buildApiClassLoader(File apiLibDirectory) { + /* + * This can just use the built-in classloading, which checks the parent classloader first and then checks its own + * classpath. A null parent means bootstrap classloader, which contains core Java classes (e.g. java.lang.String). + * This doesn't need to be isolated from the parent, because we only want to load all bootstrap classes from the + * bootstrap classloader. + */ + return new URLClassLoader(getClasspathAsURLs(apiLibDirectory), null); + } + + /** + * Build the {@link ClassLoader} which can load application classes. + * + * This sets up the link between the application classloader and the API classloader (see {@link #buildClassLoader()}. + */ + private static ClassLoader buildApplicationClassLoader(File applicationLibDirectory, File apiLibDirectory, + ClassLoader apiClassLoader) { + return LoaderBuilder.anIsolatingLoader() + // look in application lib directory for JARs + .withClasspath(getClasspathAsURIs(applicationLibDirectory)) + // getClasspathAsURIs should only return JARs within applicationLibDirectory anyways, but doing it to be safe + .withOriginRestriction(OriginRestriction.denyByDefault().allowingDirectory(applicationLibDirectory, false)) + // delegate to the api classloader for API classes + .withParentRelationship(buildApiParentRelationship(apiLibDirectory, apiClassLoader)) + .build(); + } + + /** + * Build the {@link ClassLoader} which can load Samza framework core classes. + * This may also fall back to loading application classes. + * + * This sets up two links: One link between the infrastructure classloader and the API and another link between the + * infrastructure classloader and the application classloader (see {@link #buildClassLoader()}. + */ + private static ClassLoader buildInfrastructureClassLoader(File infrastructureLibDirectory, File apiLibDirectory, + ClassLoader apiClassLoader, ClassLoader applicationClassLoader) { + return LoaderBuilder.anIsolatingLoader() + // look in infrastructure lib directory for JARs + .withClasspath(getClasspathAsURIs(infrastructureLibDirectory)) + // getClasspathAsURIs should only return JARs within infrastructureLibDirectory anyways, but doing it to be safe + .withOriginRestriction(OriginRestriction.denyByDefault().allowingDirectory(infrastructureLibDirectory, false)) + .withParentRelationship(buildApiParentRelationship(apiLibDirectory, apiClassLoader)) + /* + * Fall back to the application classloader for certain classes. For example, the application might implement + * some pluggable classes (e.g. SystemFactory). Another example is message schemas that are supplied by the + * application. + */ + .addFallbackDelegate(DelegateRelationshipBuilder.builder() + .withDelegateClassLoader(applicationClassLoader) + /* + * NONE means that a class will be loaded from here if it is not found in the classpath of the loader that uses + * this relationship. + */ + .withIsolationLevel(IsolationLevel.NONE) + .build()) + .build(); + } + + /** + * Build a {@link DelegateRelationship} which defines how to delegate to the API classloader. + * + * Delegation will only happen for classes specified in + * {@link DependencyIsolationUtils#FRAMEWORK_API_CLASS_LIST_FILE_NAME} and the Java bootstrap classes. + */ + private static DelegateRelationship buildApiParentRelationship(File apiLibDirectory, ClassLoader apiClassLoader) { + DelegateRelationshipBuilder apiParentRelationshipBuilder = DelegateRelationshipBuilder.builder() + // needs to load API classes from the API classloader + .withDelegateClassLoader(apiClassLoader) + /* + * FULL means to only load classes explicitly specified as "API" from the API classloader. We will use + * delegate-preferred class predicates to specify which classes are "API" (see below). + */ + .withIsolationLevel(IsolationLevel.FULL); + + // bootstrap classes need to be loaded from a common classloader + apiParentRelationshipBuilder.addDelegatePreferredClassPredicate(new BootstrapClassPredicate()); + // the classes which are Samza framework API classes are added here + getFrameworkApiClassGlobs(apiLibDirectory).forEach( + apiClassName -> apiParentRelationshipBuilder.addDelegatePreferredClassPredicate(new GlobMatcher(apiClassName))); + return apiParentRelationshipBuilder.build(); + } + + /** + * Gets the globs for matching against classes to load from the framework API classloader. This will read the + * {@link DependencyIsolationUtils#FRAMEWORK_API_CLASS_LIST_FILE_NAME} file in {@code directoryWithClassList} to get + * the globs. + * + * @param directoryWithClassList Directory in which + * {@link DependencyIsolationUtils#FRAMEWORK_API_CLASS_LIST_FILE_NAME} lives + * @return {@link List} of globs for matching against classes to load from the framework API classloader + */ + @VisibleForTesting + static List getFrameworkApiClassGlobs(File directoryWithClassList) { + File parentPreferredFile = + new File(directoryWithClassList, DependencyIsolationUtils.FRAMEWORK_API_CLASS_LIST_FILE_NAME); + validateCanAccess(parentPreferredFile); + try { + return Files.readAllLines(Paths.get(parentPreferredFile.toURI()), StandardCharsets.UTF_8) + .stream() + .filter(StringUtils::isNotBlank) + .collect(Collectors.toList()); + } catch (IOException e) { + throw new SamzaException("Error while reading samza-api class list", e); + } + } + + /** + * Get the {@link URL}s of all JARs/WARs in the directory {@code jarsLocation}. This only looks one level down; it is + * not recursive. + */ + @VisibleForTesting + static URL[] getClasspathAsURLs(File jarsLocation) { + validateCanAccess(jarsLocation); + File[] filesInJarsLocation = jarsLocation.listFiles(); + if (filesInJarsLocation == null) { + throw new SamzaException( + String.format("Could not find any files inside %s, probably because it is not a directory", + jarsLocation.getPath())); + } + URL[] urls = Stream.of(filesInJarsLocation) + .filter(file -> file.getName().endsWith(".jar") || file.getName().endsWith(".war")) + .map(IsolatingClassLoaderFactory::fileURL) + .toArray(URL[]::new); + LOG.info("Found {} items to load into classpath from {}", urls.length, jarsLocation); + Stream.of(urls).forEach(url -> LOG.debug("Found {} from {}", url, jarsLocation)); + return urls; + } + + /** + * Get the {@link URI}s of all JARs/WARs in the directory {@code jarsLocation}. This only looks one level down; it is + * not recursive. + */ + @VisibleForTesting + static List getClasspathAsURIs(File jarsLocation) { + return Stream.of(getClasspathAsURLs(jarsLocation)) + .map(IsolatingClassLoaderFactory::urlToURI) + .collect(Collectors.toList()); + } + + /** + * Makes sure that a file exists and can be read. + */ + private static void validateCanAccess(File file) { + if (!file.exists() || !file.canRead()) { + throw new SamzaException("Unable to access file: " + file); + } + } + + /** + * Get the {@link URL} for a {@link File}. + * Converts checked exceptions into {@link SamzaException}s. + */ + private static URL fileURL(File file) { + URI uri = file.toURI(); + try { + return uri.toURL(); + } catch (MalformedURLException e) { + throw new SamzaException("Unable to get URL for file: " + file, e); + } + } + + /** + * Get the {@link URI} for a {@link URL}. + * Converts checked exceptions into {@link SamzaException}s. + */ + private static URI urlToURI(URL url) { + try { + return url.toURI(); + } catch (URISyntaxException e) { + throw new SamzaException("Unable to get URI for URL: " + url, e); + } + } + + /** + * Get the {@link File} representing the {@link #LIB_DIRECTORY} inside the given {@code file}. + */ + private static File libDirectory(File file) { + return new File(file, LIB_DIRECTORY); + } +} diff --git a/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterBasedJobCoordinator.java b/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterBasedJobCoordinator.java index a308c19572..d9ca4d5ac6 100644 --- a/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterBasedJobCoordinator.java +++ b/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterBasedJobCoordinator.java @@ -20,6 +20,8 @@ import com.google.common.annotations.VisibleForTesting; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -27,6 +29,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; import org.apache.samza.SamzaException; +import org.apache.samza.classloader.IsolatingClassLoaderFactory; import org.apache.samza.config.ClusterManagerConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; @@ -403,10 +406,80 @@ ContainerProcessManager createContainerProcessManager() { } /** - * The entry point for the {@link ClusterBasedJobCoordinator} - * @param args args + * The entry point for the {@link ClusterBasedJobCoordinator}. */ public static void main(String[] args) { + boolean dependencyIsolationEnabled = Boolean.parseBoolean( + System.getenv(ShellCommandConfig.ENV_CLUSTER_BASED_JOB_COORDINATOR_DEPENDENCY_ISOLATION_ENABLED())); + if (!dependencyIsolationEnabled) { + // no isolation enabled, so can just execute runClusterBasedJobCoordinator directly + runClusterBasedJobCoordinator(args); + } else { + runWithClassLoader(new IsolatingClassLoaderFactory().buildClassLoader(), args); + } + } + + /** + * Execute the coordinator using a separate isolated classloader. + * @param classLoader {@link ClassLoader} to use to load the {@link ClusterBasedJobCoordinator} which will run + * @param args arguments to pass when running the {@link ClusterBasedJobCoordinator} + */ + @VisibleForTesting + static void runWithClassLoader(ClassLoader classLoader, String[] args) { + // need to use the isolated classloader to load ClusterBasedJobCoordinator and then run using that new class + Class clusterBasedJobCoordinatorClass; + try { + clusterBasedJobCoordinatorClass = classLoader.loadClass(ClusterBasedJobCoordinator.class.getName()); + } catch (ClassNotFoundException e) { + throw new SamzaException( + "Isolation was enabled, but unable to find ClusterBasedJobCoordinator in isolated classloader", e); + } + + // save the current context classloader so it can be reset after finishing the call to runClusterBasedJobCoordinator + ClassLoader previousContextClassLoader = Thread.currentThread().getContextClassLoader(); + // this is needed because certain libraries (e.g. log4j) use the context classloader + Thread.currentThread().setContextClassLoader(classLoader); + + try { + executeRunClusterBasedJobCoordinatorForClass(clusterBasedJobCoordinatorClass, args); + } finally { + // reset the context class loader; it's good practice, and could be important when running a test suite + Thread.currentThread().setContextClassLoader(previousContextClassLoader); + } + } + + /** + * Runs the {@link ClusterBasedJobCoordinator#runClusterBasedJobCoordinator(String[])} method of the given + * {@code clusterBasedJobCoordinatorClass} using reflection. + * @param clusterBasedJobCoordinatorClass {@link ClusterBasedJobCoordinator} {@link Class} for which to execute + * {@link ClusterBasedJobCoordinator#runClusterBasedJobCoordinator(String[])} + * @param args arguments to pass to {@link ClusterBasedJobCoordinator#runClusterBasedJobCoordinator(String[])} + */ + private static void executeRunClusterBasedJobCoordinatorForClass(Class clusterBasedJobCoordinatorClass, + String[] args) { + Method runClusterBasedJobCoordinatorMethod; + try { + runClusterBasedJobCoordinatorMethod = + clusterBasedJobCoordinatorClass.getDeclaredMethod("runClusterBasedJobCoordinator", String[].class); + } catch (NoSuchMethodException e) { + throw new SamzaException("Isolation was enabled, but unable to find runClusterBasedJobCoordinator method", e); + } + // only sets accessible flag for this Method instance, not other Method instances for runClusterBasedJobCoordinator + runClusterBasedJobCoordinatorMethod.setAccessible(true); + + try { + // wrapping args in object array so that args is passed as a single argument to the method + runClusterBasedJobCoordinatorMethod.invoke(null, new Object[]{args}); + } catch (IllegalAccessException | InvocationTargetException e) { + throw new SamzaException("Exception while executing runClusterBasedJobCoordinator method", e); + } + } + + /** + * This is the actual execution for the {@link ClusterBasedJobCoordinator}. This is separated out from + * {@link #main(String[])} so that it can be executed directly or from a separate classloader. + */ + private static void runClusterBasedJobCoordinator(String[] args) { Config coordinatorSystemConfig; final String coordinatorSystemEnv = System.getenv(ShellCommandConfig.ENV_COORDINATOR_SYSTEM_CONFIG()); try { diff --git a/samza-core/src/test/java/org/apache/samza/classloader/TestIsolatingClassLoaderFactory.java b/samza-core/src/test/java/org/apache/samza/classloader/TestIsolatingClassLoaderFactory.java new file mode 100644 index 0000000000..7444fbf542 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/classloader/TestIsolatingClassLoaderFactory.java @@ -0,0 +1,96 @@ +/* + * 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.classloader; + +import java.io.File; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.Paths; +import java.util.List; +import java.util.Set; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.samza.SamzaException; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + + +public class TestIsolatingClassLoaderFactory { + @Test + public void testGetApiClasses() throws URISyntaxException { + File apiClassListFile = Paths.get(getClass().getResource("/classloader").toURI()).toFile(); + List apiClassNames = IsolatingClassLoaderFactory.getFrameworkApiClassGlobs(apiClassListFile); + List expected = ImmutableList.of( + "org.apache.samza.JavaClass", + "org.apache.samza.JavaClass$InnerJavaClass", + "org.apache.samza.ScalaClass$", + "org.apache.samza.ScalaClass$$anon$1", + "my.package.with.wildcard.*", + "my.package.with.question.mark?"); + assertEquals(expected, apiClassNames); + } + + @Test(expected = SamzaException.class) + public void testGetApiClassesFileDoesNotExist() throws URISyntaxException { + File nonExistentDirectory = + new File(Paths.get(getClass().getResource("/classloader").toURI()).toFile(), "doesNotExist"); + IsolatingClassLoaderFactory.getFrameworkApiClassGlobs(nonExistentDirectory); + } + + @Test + public void testGetClasspathAsURLs() throws URISyntaxException { + File classpathDirectory = Paths.get(getClass().getResource("/classloader/classpath").toURI()).toFile(); + URL[] classpath = IsolatingClassLoaderFactory.getClasspathAsURLs(classpathDirectory); + assertEquals(2, classpath.length); + Set classpathSet = ImmutableSet.copyOf(classpath); + URL jarUrl = getClass().getResource("/classloader/classpath/placeholder-jar.jar"); + assertTrue(classpathSet.contains(jarUrl)); + URL warUrl = getClass().getResource("/classloader/classpath/placeholder-war.war"); + assertTrue(classpathSet.contains(warUrl)); + } + + @Test(expected = SamzaException.class) + public void testGetClasspathAsURLsDirectoryDoesNotExist() throws URISyntaxException { + File nonExistentDirectory = + new File(Paths.get(getClass().getResource("/classloader").toURI()).toFile(), "doesNotExist"); + IsolatingClassLoaderFactory.getClasspathAsURLs(nonExistentDirectory); + } + + @Test + public void testGetClasspathAsURIs() throws URISyntaxException { + File classpathDirectory = Paths.get(getClass().getResource("/classloader/classpath").toURI()).toFile(); + List classpath = IsolatingClassLoaderFactory.getClasspathAsURIs(classpathDirectory); + assertEquals(2, classpath.size()); + Set classpathSet = ImmutableSet.copyOf(classpath); + URL jarUrl = getClass().getResource("/classloader/classpath/placeholder-jar.jar"); + assertTrue(classpathSet.contains(jarUrl.toURI())); + URL warUrl = getClass().getResource("/classloader/classpath/placeholder-war.war"); + assertTrue(classpathSet.contains(warUrl.toURI())); + } + + @Test(expected = SamzaException.class) + public void testGetClasspathAsURIsDirectoryDoesNotExist() throws URISyntaxException { + File nonExistentDirectory = + new File(Paths.get(getClass().getResource("/classloader").toURI()).toFile(), "doesNotExist"); + IsolatingClassLoaderFactory.getClasspathAsURIs(nonExistentDirectory); + } +} \ No newline at end of file diff --git a/samza-core/src/test/java/org/apache/samza/clustermanager/TestClusterBasedJobCoordinator.java b/samza-core/src/test/java/org/apache/samza/clustermanager/TestClusterBasedJobCoordinator.java index ec79f3cb7c..8c51c642f0 100644 --- a/samza-core/src/test/java/org/apache/samza/clustermanager/TestClusterBasedJobCoordinator.java +++ b/samza-core/src/test/java/org/apache/samza/clustermanager/TestClusterBasedJobCoordinator.java @@ -47,21 +47,25 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; +import static org.mockito.AdditionalMatchers.aryEq; import static org.mockito.Matchers.anyObject; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.verifyPrivate; + /** * Tests for {@link ClusterBasedJobCoordinator} */ @RunWith(PowerMockRunner.class) -@PrepareForTest(CoordinatorStreamUtil.class) +@PrepareForTest({CoordinatorStreamUtil.class, ClusterBasedJobCoordinator.class}) public class TestClusterBasedJobCoordinator { Map configMap; @@ -162,4 +166,36 @@ public void testVerifyStartpointManagerFanOut() throws IOException { } fail("Expected run() method to stop after StartpointManager#stop()"); } + + @Test + public void testRunWithClassLoader() throws Exception { + // partially mock ClusterBasedJobCoordinator (mock runClusterBasedJobCoordinator method only) + PowerMockito.spy(ClusterBasedJobCoordinator.class); + // save the context classloader to make sure that it gets set properly once the test is finished + ClassLoader previousContextClassLoader = Thread.currentThread().getContextClassLoader(); + ClassLoader classLoader = mock(ClassLoader.class); + String[] args = new String[]{"arg0", "arg1"}; + doReturn(ClusterBasedJobCoordinator.class).when(classLoader).loadClass(ClusterBasedJobCoordinator.class.getName()); + + // stub the private static method which is called by reflection + PowerMockito.doAnswer(invocation -> { + // make sure the only calls to this method has the expected arguments + assertArrayEquals(args, invocation.getArgumentAt(0, String[].class)); + // checks that the context classloader is set correctly + assertEquals(classLoader, Thread.currentThread().getContextClassLoader()); + return null; + }).when(ClusterBasedJobCoordinator.class, "runClusterBasedJobCoordinator", any()); + + try { + ClusterBasedJobCoordinator.runWithClassLoader(classLoader, args); + assertEquals(previousContextClassLoader, Thread.currentThread().getContextClassLoader()); + } finally { + // reset it explicitly just in case runWithClassLoader throws an exception + Thread.currentThread().setContextClassLoader(previousContextClassLoader); + } + // make sure that the classloader got used + verify(classLoader).loadClass(ClusterBasedJobCoordinator.class.getName()); + // make sure runClusterBasedJobCoordinator only got called once + verifyPrivate(ClusterBasedJobCoordinator.class).invoke("runClusterBasedJobCoordinator", new Object[]{aryEq(args)}); + } } diff --git a/samza-core/src/test/resources/classloader/classpath/placeholder-jar.jar b/samza-core/src/test/resources/classloader/classpath/placeholder-jar.jar new file mode 100644 index 0000000000..d9b2beb807 --- /dev/null +++ b/samza-core/src/test/resources/classloader/classpath/placeholder-jar.jar @@ -0,0 +1 @@ +NOT AN ACTUAL JAR; JUST A PLACEHOLDER FOR A TEST diff --git a/samza-core/src/test/resources/classloader/classpath/placeholder-txt.json b/samza-core/src/test/resources/classloader/classpath/placeholder-txt.json new file mode 100644 index 0000000000..c4336ea044 --- /dev/null +++ b/samza-core/src/test/resources/classloader/classpath/placeholder-txt.json @@ -0,0 +1 @@ +{"key": "This is a placeholder file which should be ignored in testing, since it is not a jar/war."} \ No newline at end of file diff --git a/samza-core/src/test/resources/classloader/classpath/placeholder-war.war b/samza-core/src/test/resources/classloader/classpath/placeholder-war.war new file mode 100644 index 0000000000..9fa2312946 --- /dev/null +++ b/samza-core/src/test/resources/classloader/classpath/placeholder-war.war @@ -0,0 +1 @@ +NOT AN ACTUAL WAR; JUST A PLACEHOLDER FOR A TEST diff --git a/samza-core/src/test/resources/classloader/samza-framework-api-classes.txt b/samza-core/src/test/resources/classloader/samza-framework-api-classes.txt new file mode 100644 index 0000000000..8b081d75da --- /dev/null +++ b/samza-core/src/test/resources/classloader/samza-framework-api-classes.txt @@ -0,0 +1,6 @@ +org.apache.samza.JavaClass +org.apache.samza.JavaClass$InnerJavaClass +org.apache.samza.ScalaClass$ +org.apache.samza.ScalaClass$$anon$1 +my.package.with.wildcard.* +my.package.with.question.mark? \ No newline at end of file diff --git a/samza-shell/src/main/bash/run-class.sh b/samza-shell/src/main/bash/run-class.sh index 2b7d22ee83..8cdac30456 100755 --- a/samza-shell/src/main/bash/run-class.sh +++ b/samza-shell/src/main/bash/run-class.sh @@ -39,10 +39,18 @@ fi HADOOP_YARN_HOME="${HADOOP_YARN_HOME:-$HOME/.samza}" HADOOP_CONF_DIR="${HADOOP_CONF_DIR:-$HADOOP_YARN_HOME/conf}" GC_LOG_ROTATION_OPTS="-XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=10241024" -DEFAULT_LOG4J_FILE=$base_dir/lib/log4j.xml -DEFAULT_LOG4J2_FILE=$base_dir/lib/log4j2.xml +LOG4J_FILE_NAME="log4j.xml" +LOG4J2_FILE_NAME="log4j2.xml" BASE_LIB_DIR="$base_dir/lib" +DEFAULT_LOG4J_FILE=$BASE_LIB_DIR/$LOG4J_FILE_NAME +DEFAULT_LOG4J2_FILE=$BASE_LIB_DIR/$LOG4J2_FILE_NAME +# APPLICATION_LIB_DIR can be a directory which is different from $BASE_LIB_DIR which contains some additional +# application-specific resources. If it is not set, then $BASE_LIB_DIR will be used as the value. +APPLICATION_LIB_DIR="${APPLICATION_LIB_DIR:-$BASE_LIB_DIR}" +export APPLICATION_LIB_DIR=$APPLICATION_LIB_DIR + +echo APPLICATION_LIB_DIR=$APPLICATION_LIB_DIR echo BASE_LIB_DIR=$BASE_LIB_DIR CLASSPATH="" @@ -100,11 +108,28 @@ function check_and_enable_64_bit_mode { # Make the MDC inheritable to child threads by setting the system property to true if config not explicitly specified [[ $JAVA_OPTS != *-DisThreadContextMapInheritable* ]] && JAVA_OPTS="$JAVA_OPTS -DisThreadContextMapInheritable=true" -# Check if log4j configuration is specified. If not - set to lib/log4j.xml -if [[ -n $(find "$base_dir/lib" -regex ".*samza-log4j2.*.jar*") ]]; then - [[ $JAVA_OPTS != *-Dlog4j.configurationFile* ]] && export JAVA_OPTS="$JAVA_OPTS -Dlog4j.configurationFile=file:$DEFAULT_LOG4J2_FILE" -elif [[ -n $(find "$base_dir/lib" -regex ".*samza-log4j.*.jar*") ]]; then - [[ $JAVA_OPTS != *-Dlog4j.configuration* ]] && export JAVA_OPTS="$JAVA_OPTS -Dlog4j.configuration=file:$DEFAULT_LOG4J_FILE" +# Check if log4j configuration is specified; if not, look for a configuration file: +# 1) Check if using log4j or log4j2 +# 2) Check if configuration file system property is already set +# 3) If not, then look in $APPLICATION_LIB_DIR for configuration file (remember that $APPLICATION_LIB_DIR can be same or +# different from $BASE_LIB_DIR). +# 4) If still can't find it, fall back to default (from $BASE_LIB_DIR). +if [[ -n $(find "$BASE_LIB_DIR" -regex ".*samza-log4j2.*.jar*") ]]; then + if [[ $JAVA_OPTS != *-Dlog4j.configurationFile* ]]; then + if [[ -n $(find "$APPLICATION_LIB_DIR" -maxdepth 1 -name $LOG4J2_FILE_NAME) ]]; then + export JAVA_OPTS="$JAVA_OPTS -Dlog4j.configurationFile=file:$APPLICATION_LIB_DIR/$LOG4J2_FILE_NAME" + else + export JAVA_OPTS="$JAVA_OPTS -Dlog4j.configurationFile=file:$DEFAULT_LOG4J2_FILE" + fi + fi +elif [[ -n $(find "$BASE_LIB_DIR" -regex ".*samza-log4j.*.jar*") ]]; then + if [[ $JAVA_OPTS != *-Dlog4j.configuration* ]]; then + if [[ -n $(find "$APPLICATION_LIB_DIR" -maxdepth 1 -name $LOG4J_FILE_NAME) ]]; then + export JAVA_OPTS="$JAVA_OPTS -Dlog4j.configuration=file:$APPLICATION_LIB_DIR/$LOG4J_FILE_NAME" + else + export JAVA_OPTS="$JAVA_OPTS -Dlog4j.configuration=file:$DEFAULT_LOG4J_FILE" + fi + fi fi # Check if samza.log.dir is specified. If not - set to environment variable if it is set From d2e03757905b5a7cd88670cdf9083d3441dbea39 Mon Sep 17 00:00:00 2001 From: Hai Lu Date: Tue, 26 Nov 2019 15:01:27 -0800 Subject: [PATCH 10/49] update release instruction to include integration tests (#1224) --- RELEASE.md | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/RELEASE.md b/RELEASE.md index cd2a304478..09d67cee69 100644 --- a/RELEASE.md +++ b/RELEASE.md @@ -37,6 +37,13 @@ Validate Samza using all our supported build matrix. ./bin/check-all.sh ``` +Run integration tests for YARN and standalone + +```bash + ./bin/integration-tests.sh . yarn-integration-tests + ./bin/integration-tests.sh . standalone-integration-tests +``` + To release to a local Maven repository: ```bash @@ -97,7 +104,7 @@ Upload the build artifacts to your Apache home directory: bye ``` -Make a signed git tag for the release candidate: +Make a signed git tag for the release candidate (you may need to use -u to specify key id): ```bash git tag -s release-$VERSION-rc0 -m "Apache Samza $VERSION release candidate 0" From 069a8c0912d57301b92b76d220ddbf0cca386c16 Mon Sep 17 00:00:00 2001 From: Hai Lu Date: Tue, 26 Nov 2019 17:15:44 -0800 Subject: [PATCH 11/49] SAMZA-2403: fix standalone integration failure (#1225) --- samza-test/src/main/python/tests/zk_client.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/samza-test/src/main/python/tests/zk_client.py b/samza-test/src/main/python/tests/zk_client.py index f5716e40b4..a1cf525a80 100644 --- a/samza-test/src/main/python/tests/zk_client.py +++ b/samza-test/src/main/python/tests/zk_client.py @@ -31,7 +31,7 @@ class ZkClient: Instantiates a kazoo client to connect to zookeeper server at :param zookeeper_host::param zookeeper_port. """ def __init__(self, zookeeper_host, zookeeper_port, app_name, app_id): - self.protocol_version = "1.0" + self.protocol_version = "2.0" self.kazoo_client = KazooClient(hosts='{0}:{1}'.format(zookeeper_host, zookeeper_port)) self.zk_base_node = 'app-{0}-{1}/{2}-{3}-{4}-coordinationData'.format(app_name, app_id, app_name, app_id, self.protocol_version) @@ -62,7 +62,7 @@ def get_latest_job_model(self): childZkNodes.sort() childZkNodes.reverse() - job_model_generation_path = '{0}/jobModelGeneration/jobModels/{1}/'.format(self.zk_base_node, childZkNodes[0]) + job_model_generation_path = '{0}/jobModelGeneration/jobModels/{1}/0'.format(self.zk_base_node, childZkNodes[0]) job_model, _ = self.kazoo_client.get(job_model_generation_path) """ @@ -75,7 +75,8 @@ def get_latest_job_model(self): """ first_curly_brace_index = job_model.find('{') - job_model = job_model[first_curly_brace_index: ] + last_curly_brace_index = job_model.rindex('}') + job_model = job_model[first_curly_brace_index: last_curly_brace_index + 1] job_model_dict = json.loads(job_model) logger.info("Recent JobModel in zookeeper: {0}".format(job_model_dict)) except: From 20eb3456f56f0c32f463588e849263cecf683e7d Mon Sep 17 00:00:00 2001 From: Cameron Lee Date: Mon, 2 Dec 2019 10:39:39 -0800 Subject: [PATCH 12/49] SAMZA-2388: [Scala cleanup] Convert Util to Java (#1221) --- .../org/apache/samza/config/SystemConfig.java | 4 +- .../samza/execution/LocalJobPlanner.java | 5 +- .../samza/runtime/LocalApplicationRunner.java | 4 +- .../org/apache/samza/util/ConfigUtil.java | 54 ++--- .../apache/samza/util/DiagnosticsUtil.java | 2 +- .../main/java/org/apache/samza/util/Util.java | 123 +++++++++++ .../samza/checkpoint/CheckpointTool.scala | 4 +- .../samza/coordinator/JobModelManager.scala | 4 +- .../samza/diagnostics/DiagnosticsManager.java | 29 +-- .../MetricsSnapshotReporterFactory.scala | 6 +- .../org/apache/samza/util/CommandLine.scala | 8 +- .../samza/util/CoordinatorStreamUtil.scala | 2 +- .../org/apache/samza/util/FileUtil.scala | 2 - .../org/apache/samza/util/HttpUtil.scala | 3 +- .../scala/org/apache/samza/util/Util.scala | 180 ---------------- .../org/apache/samza/util/TestConfigUtil.java | 182 +++++++++++++--- .../java/org/apache/samza/util/TestUtil.java | 201 ++++++++++++++++++ .../org/apache/samza/util/TestUtil.scala | 44 ---- .../kafka/KafkaCheckpointManagerFactory.scala | 4 +- .../validation/YarnJobValidationTool.java | 10 +- .../scalate/WEB-INF/views/index.scaml | 2 +- 21 files changed, 547 insertions(+), 326 deletions(-) create mode 100644 samza-core/src/main/java/org/apache/samza/util/Util.java delete mode 100644 samza-core/src/main/scala/org/apache/samza/util/Util.scala create mode 100644 samza-core/src/test/java/org/apache/samza/util/TestUtil.java delete mode 100644 samza-core/src/test/scala/org/apache/samza/util/TestUtil.scala diff --git a/samza-core/src/main/java/org/apache/samza/config/SystemConfig.java b/samza-core/src/main/java/org/apache/samza/config/SystemConfig.java index bfbc2979e9..7b44a3a08f 100644 --- a/samza-core/src/main/java/org/apache/samza/config/SystemConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/SystemConfig.java @@ -29,7 +29,7 @@ import org.apache.samza.SamzaException; import org.apache.samza.system.SystemAdmin; import org.apache.samza.system.SystemFactory; -import org.apache.samza.util.Util; +import org.apache.samza.util.ReflectionUtil; /** * Config helper methods related to systems. @@ -117,7 +117,7 @@ public Map getSystemFactories() { systemName -> { String systemFactoryClassName = getSystemFactory(systemName).orElseThrow(() -> new SamzaException( String.format("A stream uses system %s, which is missing from the configuration.", systemName))); - return Util.getObj(systemFactoryClassName, SystemFactory.class); + return ReflectionUtil.getObj(systemFactoryClassName, SystemFactory.class); })); return systemFactories; diff --git a/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java b/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java index 902ea50fc9..000e55a4e0 100644 --- a/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java +++ b/samza-core/src/main/java/org/apache/samza/execution/LocalJobPlanner.java @@ -36,7 +36,7 @@ import org.apache.samza.metadatastore.MetadataStoreFactory; import org.apache.samza.metrics.MetricsRegistryMap; import org.apache.samza.system.StreamSpec; -import org.apache.samza.util.Util; +import org.apache.samza.util.ReflectionUtil; import org.apache.samza.zk.ZkMetadataStoreFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -210,7 +210,8 @@ private MetadataStore getMetadataStore() { if (metadataStoreFactoryClass == null) { metadataStoreFactoryClass = DEFAULT_METADATA_STORE_FACTORY; } - MetadataStoreFactory metadataStoreFactory = Util.getObj(metadataStoreFactoryClass, MetadataStoreFactory.class); + MetadataStoreFactory metadataStoreFactory = + ReflectionUtil.getObj(metadataStoreFactoryClass, MetadataStoreFactory.class); return metadataStoreFactory.getMetadataStore(STREAM_CREATION_METADATA_STORE, appDesc.getConfig(), new MetricsRegistryMap()); } } diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index 44adfde3a9..bf6dfce2d8 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -64,7 +64,6 @@ import org.apache.samza.task.TaskFactoryUtil; import org.apache.samza.util.CoordinatorStreamUtil; import org.apache.samza.util.ReflectionUtil; -import org.apache.samza.util.Util; import org.apache.samza.zk.ZkJobCoordinatorFactory; import org.apache.samza.zk.ZkMetadataStoreFactory; import org.slf4j.Logger; @@ -384,7 +383,8 @@ private void cleanup() { */ private MetadataStore getMetadataStoreForRunID() { String metadataStoreFactoryClass = appDesc.getConfig().getOrDefault(METADATA_STORE_FACTORY_CONFIG, DEFAULT_METADATA_STORE_FACTORY); - MetadataStoreFactory metadataStoreFactory = Util.getObj(metadataStoreFactoryClass, MetadataStoreFactory.class); + MetadataStoreFactory metadataStoreFactory = + ReflectionUtil.getObj(metadataStoreFactoryClass, MetadataStoreFactory.class); return metadataStoreFactory.getMetadataStore(RUN_ID_METADATA_STORE, appDesc.getConfig(), new MetricsRegistryMap()); } diff --git a/samza-core/src/main/java/org/apache/samza/util/ConfigUtil.java b/samza-core/src/main/java/org/apache/samza/util/ConfigUtil.java index 8567ecdb7e..7d86bf52a0 100644 --- a/samza-core/src/main/java/org/apache/samza/util/ConfigUtil.java +++ b/samza-core/src/main/java/org/apache/samza/util/ConfigUtil.java @@ -19,16 +19,18 @@ package org.apache.samza.util; -import java.util.HashMap; -import java.util.Map; +import java.util.Optional; import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigRewriter; import org.apache.samza.config.JobConfig; -import org.apache.samza.config.MapConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ConfigUtil { + private static final Logger LOG = LoggerFactory.getLogger(ConfigUtil.class); + /** * Re-writes configuration using a ConfigRewriter, if one is defined. If * there is no ConfigRewriter defined for the job, then this method is a @@ -37,28 +39,32 @@ public class ConfigUtil { * @param config The config to re-write * @return rewrited configs */ - static public Config rewriteConfig(Config config) { - try { - final String rewriters = config.get(JobConfig.CONFIG_REWRITERS, ""); - if (!rewriters.isEmpty()) { - Map resultConfig = new HashMap<>(config); - for (String rewriter : rewriters.split(",")) { - String rewriterClassCfg = String.format(JobConfig.CONFIG_REWRITER_CLASS, rewriter); - String rewriterClass = config.get(rewriterClassCfg, ""); - if (rewriterClass.isEmpty()) { - throw new SamzaException( - "Unable to find class config for config rewriter: " + rewriterClassCfg); - } - ConfigRewriter configRewriter = (ConfigRewriter) Class.forName(rewriterClass).newInstance(); - Config rewritedConfig = configRewriter.rewrite(rewriter, config); - resultConfig.putAll(rewritedConfig); - } - return new MapConfig(resultConfig); - } else { - return config; + public static Config rewriteConfig(Config config) { + Optional configRewriterNamesOptional = new JobConfig(config).getConfigRewriters(); + if (configRewriterNamesOptional.isPresent()) { + String[] configRewriterNames = configRewriterNamesOptional.get().split(","); + Config rewrittenConfig = config; + for (String configRewriterName : configRewriterNames) { + rewrittenConfig = applyRewriter(rewrittenConfig, configRewriterName); } - } catch (Exception e) { - throw new RuntimeException(e); + return rewrittenConfig; + } else { + return config; } } + + /** + * Re-writes configuration using a ConfigRewriter, defined with the given rewriterName in config. + * @param config the config to re-write + * @param rewriterName the name of the rewriter to apply + * @return the rewritten config + */ + public static Config applyRewriter(Config config, String rewriterName) { + String rewriterClassName = new JobConfig(config).getConfigRewriterClass(rewriterName) + .orElseThrow(() -> new SamzaException( + String.format("Unable to find class config for config rewriter %s.", rewriterName))); + ConfigRewriter rewriter = ReflectionUtil.getObj(rewriterClassName, ConfigRewriter.class); + LOG.info("Re-writing config with {}", rewriter); + return rewriter.rewrite(rewriterName, config); + } } diff --git a/samza-core/src/main/java/org/apache/samza/util/DiagnosticsUtil.java b/samza-core/src/main/java/org/apache/samza/util/DiagnosticsUtil.java index 4d2a3bcc1e..e3ff250812 100644 --- a/samza-core/src/main/java/org/apache/samza/util/DiagnosticsUtil.java +++ b/samza-core/src/main/java/org/apache/samza/util/DiagnosticsUtil.java @@ -132,7 +132,7 @@ public static Optional> buildD } // Create a systemProducer for giving to diagnostic-reporter and diagnosticsManager - SystemFactory systemFactory = Util.getObj(diagnosticsSystemFactoryName.get(), SystemFactory.class); + SystemFactory systemFactory = ReflectionUtil.getObj(diagnosticsSystemFactoryName.get(), SystemFactory.class); SystemProducer systemProducer = systemFactory.getProducer(diagnosticsSystemStream.getSystem(), config, new MetricsRegistryMap()); DiagnosticsManager diagnosticsManager = diff --git a/samza-core/src/main/java/org/apache/samza/util/Util.java b/samza-core/src/main/java/org/apache/samza/util/Util.java new file mode 100644 index 0000000000..875b6da6d9 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/util/Util.java @@ -0,0 +1,123 @@ +/* + * 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.util; + +import java.net.Inet4Address; +import java.net.InetAddress; +import java.net.NetworkInterface; +import java.net.SocketException; +import java.net.UnknownHostException; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import com.google.common.collect.Lists; +import org.apache.samza.SamzaException; +import org.apache.samza.config.ApplicationConfig; +import org.apache.samza.config.Config; +import org.apache.samza.config.TaskConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class Util { + private static final Logger LOG = LoggerFactory.getLogger(Util.class); + + static final String FALLBACK_VERSION = "0.0.1"; + + /** + * Make an environment variable string safe to pass. + */ + public static String envVarEscape(String str) { + return str.replace("\"", "\\\"").replace("'", "\\'"); + } + + public static String getSamzaVersion() { + return Optional.ofNullable(Util.class.getPackage().getImplementationVersion()).orElseGet(() -> { + LOG.warn("Unable to find implementation samza version in jar's meta info. Defaulting to {}", FALLBACK_VERSION); + return FALLBACK_VERSION; + }); + } + + public static String getTaskClassVersion(Config config) { + try { + Optional appClass = Optional.ofNullable(new ApplicationConfig(config).getAppClass()); + if (appClass.isPresent()) { + return Optional.ofNullable(Class.forName(appClass.get()).getPackage().getImplementationVersion()) + .orElse(FALLBACK_VERSION); + } else { + Optional taskClass = new TaskConfig(config).getTaskClass(); + if (taskClass.isPresent()) { + return Optional.ofNullable(Class.forName(taskClass.get()).getPackage().getImplementationVersion()) + .orElse(FALLBACK_VERSION); + } else { + LOG.warn("Unable to find app class or task class. Defaulting to {}", FALLBACK_VERSION); + return FALLBACK_VERSION; + } + } + } catch (Exception e) { + LOG.warn(String.format("Ran into exception while trying to get version of app or task. Defaulting to %s", + FALLBACK_VERSION), e); + return FALLBACK_VERSION; + } + } + + /** + * Returns the the first host address which is not the loopback address, or {@link InetAddress#getLocalHost} as a + * fallback. + * + * @return the {@link InetAddress} which represents the localhost + */ + public static InetAddress getLocalHost() { + try { + return doGetLocalHost(); + } catch (Exception e) { + throw new SamzaException("Error while getting localhost", e); + } + } + + private static InetAddress doGetLocalHost() throws UnknownHostException, SocketException { + InetAddress localHost = InetAddress.getLocalHost(); + if (localHost.isLoopbackAddress()) { + LOG.debug("Hostname {} resolves to a loopback address, trying to resolve an external IP address.", + localHost.getHostName()); + List networkInterfaces; + if (System.getProperty("os.name").startsWith("Windows")) { + networkInterfaces = Collections.list(NetworkInterface.getNetworkInterfaces()); + } else { + networkInterfaces = Lists.reverse(Collections.list(NetworkInterface.getNetworkInterfaces())); + } + for (NetworkInterface networkInterface : networkInterfaces) { + List addresses = Collections.list(networkInterface.getInetAddresses()) + .stream() + .filter(address -> !(address.isLinkLocalAddress() || address.isLoopbackAddress())) + .collect(Collectors.toList()); + if (!addresses.isEmpty()) { + InetAddress address = addresses.stream() + .filter(addr -> addr instanceof Inet4Address) + .findFirst() + .orElseGet(() -> addresses.get(0)); + LOG.debug("Found an external IP address {} which represents the localhost.", address.getHostAddress()); + return InetAddress.getByAddress(address.getAddress()); + } + } + } + return localHost; + } +} \ No newline at end of file diff --git a/samza-core/src/main/scala/org/apache/samza/checkpoint/CheckpointTool.scala b/samza-core/src/main/scala/org/apache/samza/checkpoint/CheckpointTool.scala index 74ab562b07..eff1e73955 100644 --- a/samza-core/src/main/scala/org/apache/samza/checkpoint/CheckpointTool.scala +++ b/samza-core/src/main/scala/org/apache/samza/checkpoint/CheckpointTool.scala @@ -32,7 +32,7 @@ import org.apache.samza.container.TaskName import org.apache.samza.job.JobRunner.info import org.apache.samza.metrics.MetricsRegistryMap import org.apache.samza.system.SystemStreamPartition -import org.apache.samza.util.{CommandLine, CoordinatorStreamUtil, Logging, ReflectionUtil, Util} +import org.apache.samza.util.{CommandLine, ConfigUtil, CoordinatorStreamUtil, Logging, ReflectionUtil, Util} import org.apache.samza.Partition import org.apache.samza.SamzaException @@ -138,7 +138,7 @@ object CheckpointTool { val options = cmdline.parser.parse(args: _*) val userConfig = cmdline.loadConfig(options) val jobConfig = JobPlanner.generateSingleJobConfig(userConfig) - val rewrittenConfig = Util.rewriteConfig(jobConfig) + val rewrittenConfig = ConfigUtil.rewriteConfig(jobConfig) info(s"Using the rewritten config: $rewrittenConfig") val tool = CheckpointTool(rewrittenConfig, cmdline.newOffsets) tool.run() diff --git a/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala b/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala index 76245b84ec..07055de1fc 100644 --- a/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala +++ b/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala @@ -47,7 +47,7 @@ import org.apache.samza.metrics.MetricsRegistryMap import org.apache.samza.runtime.LocationId import org.apache.samza.system._ import org.apache.samza.util.ScalaJavaUtil.JavaOptionals -import org.apache.samza.util.{Logging, ReflectionUtil, Util} +import org.apache.samza.util.{ConfigUtil, Logging, ReflectionUtil, Util} import scala.collection.JavaConverters import scala.collection.JavaConversions._ @@ -274,7 +274,7 @@ object JobModelManager extends Logging { filter(rewriterName => JavaOptionals.toRichOptional(jobConfig.getConfigRewriterClass(rewriterName)).toOption .getOrElse(throw new SamzaException("Unable to find class config for config rewriter %s." format rewriterName)) .equalsIgnoreCase(classOf[RegExTopicGenerator].getName)). - foldLeft(config)(Util.applyRewriter(_, _)) + foldLeft(config)(ConfigUtil.applyRewriter(_, _)) case _ => config } } diff --git a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsManager.java b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsManager.java index 80ddf0dc5f..0900c138fd 100644 --- a/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsManager.java +++ b/samza-core/src/main/scala/org/apache/samza/diagnostics/DiagnosticsManager.java @@ -21,11 +21,9 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import java.lang.reflect.InvocationTargetException; import java.time.Duration; import java.time.Instant; import java.util.ArrayList; -import java.util.Collections; import java.util.Map; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.Executors; @@ -36,11 +34,9 @@ import org.apache.samza.system.OutgoingMessageEnvelope; import org.apache.samza.system.SystemProducer; import org.apache.samza.system.SystemStream; -import org.apache.samza.util.Util; +import org.apache.samza.util.ReflectionUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.Tuple2; -import scala.collection.JavaConverters; /** @@ -146,24 +142,15 @@ public DiagnosticsManager(String jobName, resetTime = Instant.now(); try { - - Util.getObj("org.apache.samza.logging.log4j.SimpleDiagnosticsAppender", - JavaConverters.collectionAsScalaIterableConverter( - Collections.singletonList(new Tuple2, Object>(DiagnosticsManager.class, this))) - .asScala() - .toSeq()); - + ReflectionUtil.getObjWithArgs("org.apache.samza.logging.log4j.SimpleDiagnosticsAppender", + Object.class, ReflectionUtil.constructorArgument(this, DiagnosticsManager.class)); LOG.info("Attached log4j diagnostics appender."); - } catch (ClassNotFoundException | InstantiationException | InvocationTargetException e) { + } catch (Exception e) { try { - Util.getObj("org.apache.samza.logging.log4j2.SimpleDiagnosticsAppender", - JavaConverters.collectionAsScalaIterableConverter( - Collections.singletonList(new Tuple2, Object>(DiagnosticsManager.class, this))) - .asScala() - .toSeq()); - LOG.info("Attached log4j diagnostics appender."); - } catch (ClassNotFoundException | InstantiationException | InvocationTargetException ex) { - + ReflectionUtil.getObjWithArgs("org.apache.samza.logging.log4j2.SimpleDiagnosticsAppender", + Object.class, ReflectionUtil.constructorArgument(this, DiagnosticsManager.class)); + LOG.info("Attached log4j2 diagnostics appender."); + } catch (Exception ex) { LOG.warn( "Failed to instantiate neither diagnostic appender for sending error information to diagnostics stream.", ex); diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.scala b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.scala index ec5f4d971f..441d834f9c 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/reporter/MetricsSnapshotReporterFactory.scala @@ -19,7 +19,7 @@ package org.apache.samza.metrics.reporter -import org.apache.samza.util.{Logging, StreamUtil, Util} +import org.apache.samza.util.{Logging, ReflectionUtil, StreamUtil, Util} import org.apache.samza.SamzaException import org.apache.samza.config.{Config, JobConfig, MetricsConfig, SerializerConfig, StreamConfig, SystemConfig} import org.apache.samza.metrics.MetricsReporter @@ -53,7 +53,7 @@ class MetricsSnapshotReporterFactory extends MetricsReporterFactory with Logging val systemFactoryClassName = JavaOptionals.toRichOptional(systemConfig.getSystemFactory(systemName)).toOption .getOrElse(throw new SamzaException("Trying to fetch system factory for system %s, which isn't defined in config." format systemName)) - val systemFactory = Util.getObj(systemFactoryClassName, classOf[SystemFactory]) + val systemFactory = ReflectionUtil.getObj(systemFactoryClassName, classOf[SystemFactory]) info("Got system factory %s." format systemFactory) @@ -71,7 +71,7 @@ class MetricsSnapshotReporterFactory extends MetricsReporterFactory with Logging val serde = if (serdeName != null) { JavaOptionals.toRichOptional(serializerConfig.getSerdeFactoryClass(serdeName)).toOption match { case Some(serdeClassName) => - Util.getObj(serdeClassName, classOf[SerdeFactory[MetricsSnapshot]]).getSerde(serdeName, config) + ReflectionUtil.getObj(serdeClassName, classOf[SerdeFactory[MetricsSnapshot]]).getSerde(serdeName, config) case _ => null } } else { diff --git a/samza-core/src/main/scala/org/apache/samza/util/CommandLine.scala b/samza-core/src/main/scala/org/apache/samza/util/CommandLine.scala index 14dd1c915f..b97afad119 100644 --- a/samza-core/src/main/scala/org/apache/samza/util/CommandLine.scala +++ b/samza-core/src/main/scala/org/apache/samza/util/CommandLine.scala @@ -33,19 +33,19 @@ import scala.collection.JavaConverters._ */ class CommandLine { val parser = new OptionParser() - val configFactoryOpt = + val configFactoryOpt = parser.accepts("config-factory", "The config factory to use to read your config file.") .withRequiredArg .ofType(classOf[java.lang.String]) .describedAs("com.foo.bar.ClassName") .defaultsTo(classOf[PropertiesConfigFactory].getName) val configPathOpt = - parser.accepts("config-path", "URI location to a config file (e.g. file:///some/local/path.properties). " + + parser.accepts("config-path", "URI location to a config file (e.g. file:///some/local/path.properties). " + "If multiple files are given they are all used with later files overriding any values that appear in earlier files.") .withRequiredArg .ofType(classOf[URI]) .describedAs("path") - val configOverrideOpt = + val configOverrideOpt = parser.accepts("config", "A configuration value in the form key=value. Command line properties override any configuration values given.") .withRequiredArg .ofType(classOf[KeyValuePair]) @@ -63,7 +63,7 @@ class CommandLine { // Set up the job parameters. val configFactoryClassName = options.valueOf(configFactoryOpt) val configPaths = options.valuesOf(configPathOpt) - configFactory = Util.getObj(configFactoryClassName, classOf[ConfigFactory]) + configFactory = ReflectionUtil.getObj(configFactoryClassName, classOf[ConfigFactory]) val configOverrides = options.valuesOf(configOverrideOpt).asScala.map(kv => (kv.key, kv.value)).toMap val configs: Buffer[java.util.Map[String, String]] = configPaths.asScala.map(configFactory.getConfig) diff --git a/samza-core/src/main/scala/org/apache/samza/util/CoordinatorStreamUtil.scala b/samza-core/src/main/scala/org/apache/samza/util/CoordinatorStreamUtil.scala index 810345e005..f108387a2c 100644 --- a/samza-core/src/main/scala/org/apache/samza/util/CoordinatorStreamUtil.scala +++ b/samza-core/src/main/scala/org/apache/samza/util/CoordinatorStreamUtil.scala @@ -86,7 +86,7 @@ object CoordinatorStreamUtil extends Logging { val systemConfig = new SystemConfig(config) val systemFactoryClassName = JavaOptionals.toRichOptional(systemConfig.getSystemFactory(systemName)).toOption .getOrElse(throw new SamzaException("Missing configuration: " + SystemConfig.SYSTEM_FACTORY_FORMAT format systemName)) - Util.getObj(systemFactoryClassName, classOf[SystemFactory]) + ReflectionUtil.getObj(systemFactoryClassName, classOf[SystemFactory]) } /** diff --git a/samza-core/src/main/scala/org/apache/samza/util/FileUtil.scala b/samza-core/src/main/scala/org/apache/samza/util/FileUtil.scala index f45f28e901..d416340686 100644 --- a/samza-core/src/main/scala/org/apache/samza/util/FileUtil.scala +++ b/samza-core/src/main/scala/org/apache/samza/util/FileUtil.scala @@ -25,8 +25,6 @@ import java.io._ import java.nio.file._ import java.util.zip.CRC32 -import org.apache.samza.util.Util.info - class FileUtil extends Logging { /** * Writes checksum & data to a file diff --git a/samza-core/src/main/scala/org/apache/samza/util/HttpUtil.scala b/samza-core/src/main/scala/org/apache/samza/util/HttpUtil.scala index ea5eb5a8cc..577bba63da 100644 --- a/samza-core/src/main/scala/org/apache/samza/util/HttpUtil.scala +++ b/samza-core/src/main/scala/org/apache/samza/util/HttpUtil.scala @@ -25,9 +25,8 @@ import java.io.{BufferedReader, IOException, InputStream, InputStreamReader} import java.net.{HttpURLConnection, URL} import org.apache.samza.SamzaException -import org.apache.samza.util.Util.{error, warn} -object HttpUtil { +object HttpUtil extends Logging { /** * Reads a URL and returns the response body as a string. Retries in an exponential backoff, but does no other error handling. diff --git a/samza-core/src/main/scala/org/apache/samza/util/Util.scala b/samza-core/src/main/scala/org/apache/samza/util/Util.scala deleted file mode 100644 index 1323cd2ec6..0000000000 --- a/samza-core/src/main/scala/org/apache/samza/util/Util.scala +++ /dev/null @@ -1,180 +0,0 @@ -/* - * 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.util - - -import java.lang.reflect.InvocationTargetException - -import org.apache.samza.config._ -import org.apache.samza.SamzaException -import java.net.Inet4Address -import java.net.InetAddress -import java.net.NetworkInterface -import java.util.Random - -import org.apache.samza.util.ScalaJavaUtil.JavaOptionals - -import scala.collection.JavaConverters._ - - -object Util extends Logging { - private val FALLBACK_VERSION = "0.0.1" - val Random = new Random - - /** - * Make an environment variable string safe to pass. - */ - def envVarEscape(str: String) = str.replace("\"", "\\\"").replace("'", "\\'") - - /** - * Get a random number >= startInclusive, and < endExclusive. - */ - def randomBetween(startInclusive: Int, endExclusive: Int) = - startInclusive + Random.nextInt(endExclusive - startInclusive) - - /** - * Instantiate an object of type T from a given className. - * - * Deprecated: Use [[ReflectionUtil.getObj(String, Class)]] instead. - */ - @Deprecated - def getObj[T](className: String, clazz: Class[T]) = { - try { - Class - .forName(className) - .newInstance - .asInstanceOf[T] - } catch { - case e: Throwable => { - error("Unable to create an instance for class %s." format className, e) - throw e - } - } - } - - def getSamzaVersion(): String = { - Option(this.getClass.getPackage.getImplementationVersion) - .getOrElse({ - warn("Unable to find implementation samza version in jar's meta info. Defaulting to %s" format FALLBACK_VERSION) - FALLBACK_VERSION - }) - } - - def getTaskClassVersion(config: Config): String = { - try { - val appClass = Option(new ApplicationConfig(config).getAppClass) - if (appClass.isDefined) { - Option.apply(Class.forName(appClass.get).getPackage.getImplementationVersion).getOrElse(FALLBACK_VERSION) - } else { - val taskClass = new TaskConfig(config).getTaskClass - if (taskClass.isPresent) { - Option.apply(Class.forName(taskClass.get()).getPackage.getImplementationVersion).getOrElse(FALLBACK_VERSION) - } else { - warn("Unable to find app class or task class. Defaulting to %s" format FALLBACK_VERSION) - FALLBACK_VERSION - } - } - } catch { - case e: Exception => { - warn("Unable to find implementation version in jar's meta info. Defaulting to %s" format FALLBACK_VERSION) - FALLBACK_VERSION - } - } - } - - /** - * Instantiate an object from given className, and given constructor parameters. - * - * Deprecated: Use [[ReflectionUtil.getObjWithArgs(String, Class, ConstructorArgument...)]] instead. - */ - @Deprecated - @throws[ClassNotFoundException] - @throws[InstantiationException] - @throws[InvocationTargetException] - def getObj(className: String, constructorParams: (Class[_], Object)*) = { - try { - Class.forName(className).getDeclaredConstructor(constructorParams.map(x => x._1): _*) - .newInstance(constructorParams.map(x => x._2): _*) - } catch { - case e@(_: ClassNotFoundException | _: InstantiationException | _: InvocationTargetException) => { - warn("Could not instantiate an instance for class %s." format className, e) - throw e - } - } - } - - /** - * Returns the the first host address which is not the loopback address, or [[java.net.InetAddress#getLocalHost]] as a fallback - * - * @return the [[java.net.InetAddress]] which represents the localhost - */ - def getLocalHost: InetAddress = { - val localHost = InetAddress.getLocalHost - if (localHost.isLoopbackAddress) { - debug("Hostname %s resolves to a loopback address, trying to resolve an external IP address.".format(localHost.getHostName)) - val networkInterfaces = if (System.getProperty("os.name").startsWith("Windows")) { - NetworkInterface.getNetworkInterfaces.asScala.toList - } else { - NetworkInterface.getNetworkInterfaces.asScala.toList.reverse - } - for (networkInterface <- networkInterfaces) { - val addresses = networkInterface.getInetAddresses.asScala.toList - .filterNot(address => address.isLinkLocalAddress || address.isLoopbackAddress) - if (addresses.nonEmpty) { - val address = addresses.find(_.isInstanceOf[Inet4Address]).getOrElse(addresses.head) - debug("Found an external IP address %s which represents the localhost.".format(address.getHostAddress)) - return InetAddress.getByAddress(address.getAddress) - } - } - } - localHost - } - - /** - * Re-writes configuration using a ConfigRewriter, if one is defined. If - * there is no ConfigRewriter defined for the job, then this method is a - * no-op. - * - * @param config The config to re-write - * @return re-written config - */ - def rewriteConfig(config: Config): Config = { - JavaOptionals.toRichOptional(new JobConfig(config).getConfigRewriters).toOption match { - case Some(rewriters) => rewriters.split(",").foldLeft(config)(applyRewriter(_, _)) - case _ => config - } - } - - /** - * Re-writes configuration using a ConfigRewriter, defined with the given rewriterName in config. - * @param config the config to re-write - * @param rewriterName the name of the rewriter to apply - * @return the rewritten config - */ - def applyRewriter(config: Config, rewriterName: String): Config = { - val rewriterClassName = JavaOptionals.toRichOptional(new JobConfig(config).getConfigRewriterClass(rewriterName)) - .toOption - .getOrElse(throw new SamzaException("Unable to find class config for config rewriter %s." format rewriterName)) - val rewriter = ReflectionUtil.getObj(rewriterClassName, classOf[ConfigRewriter]) - info("Re-writing config with " + rewriter) - rewriter.rewrite(rewriterName, config) - } - -} diff --git a/samza-core/src/test/java/org/apache/samza/util/TestConfigUtil.java b/samza-core/src/test/java/org/apache/samza/util/TestConfigUtil.java index f771a99010..218eb179fd 100644 --- a/samza-core/src/test/java/org/apache/samza/util/TestConfigUtil.java +++ b/samza-core/src/test/java/org/apache/samza/util/TestConfigUtil.java @@ -21,52 +21,186 @@ import java.util.HashMap; import java.util.Map; +import com.google.common.collect.ImmutableMap; +import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.config.ConfigRewriter; +import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; -import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; public class TestConfigUtil { - Map configMap = new HashMap<>(); + private static final String CONFIG_KEY = "config.key"; + private static final String CONFIG_VALUE = "value"; + private static final String NEW_CONFIG_KEY = "new.rewritten.config.key"; + private static final String REWRITER_NAME = "propertyRewriter"; + private static final String OTHER_REWRITER_NAME = "otherPropertyRewriter"; - @Before - public void setup() { - configMap.put("job.config.rewriter.testRewriter.class", TestConfigRewriter.class.getName()); - configMap.put("job.config.rewriter.testNoneRewriter.class", ""); + @Test + public void testRewriteConfig() { + Map baseConfigMap = ImmutableMap.of(CONFIG_KEY, CONFIG_VALUE); + + // no rewriters + Map fullConfig = new HashMap<>(baseConfigMap); + assertEquals(fullConfig, ConfigUtil.rewriteConfig(new MapConfig(fullConfig))); + + // rewriter that adds property + fullConfig = new HashMap<>(baseConfigMap); + fullConfig.put(JobConfig.CONFIG_REWRITERS, REWRITER_NAME); + fullConfig.put(String.format(JobConfig.CONFIG_REWRITER_CLASS, REWRITER_NAME), NewPropertyRewriter.class.getName()); + Map expectedConfigMap = new HashMap<>(fullConfig); + expectedConfigMap.put(NEW_CONFIG_KEY, CONFIG_VALUE); + assertEquals(new MapConfig(expectedConfigMap), ConfigUtil.rewriteConfig(new MapConfig(fullConfig))); + + // rewriter that updates property + fullConfig = new HashMap<>(baseConfigMap); + fullConfig.put(JobConfig.CONFIG_REWRITERS, REWRITER_NAME); + fullConfig.put(String.format(JobConfig.CONFIG_REWRITER_CLASS, REWRITER_NAME), + UpdatePropertyRewriter.class.getName()); + expectedConfigMap = new HashMap<>(fullConfig); + expectedConfigMap.put(CONFIG_KEY, CONFIG_VALUE + CONFIG_VALUE); + assertEquals(new MapConfig(expectedConfigMap), ConfigUtil.rewriteConfig(new MapConfig(fullConfig))); + + // rewriter that removes property + fullConfig = new HashMap<>(baseConfigMap); + fullConfig.put(JobConfig.CONFIG_REWRITERS, REWRITER_NAME); + fullConfig.put(String.format(JobConfig.CONFIG_REWRITER_CLASS, REWRITER_NAME), + DeletePropertyRewriter.class.getName()); + expectedConfigMap = new HashMap<>(fullConfig); + expectedConfigMap.remove(CONFIG_KEY); + assertEquals(new MapConfig(expectedConfigMap), ConfigUtil.rewriteConfig(new MapConfig(fullConfig))); + + // only apply rewriters from rewriters list + fullConfig = new HashMap<>(baseConfigMap); + fullConfig.put(JobConfig.CONFIG_REWRITERS, OTHER_REWRITER_NAME); + fullConfig.put(String.format(JobConfig.CONFIG_REWRITER_CLASS, REWRITER_NAME), NewPropertyRewriter.class.getName()); + fullConfig.put(String.format(JobConfig.CONFIG_REWRITER_CLASS, OTHER_REWRITER_NAME), + UpdatePropertyRewriter.class.getName()); + expectedConfigMap = new HashMap<>(fullConfig); + expectedConfigMap.put(CONFIG_KEY, CONFIG_VALUE + CONFIG_VALUE); + assertEquals(new MapConfig(expectedConfigMap), ConfigUtil.rewriteConfig(new MapConfig(fullConfig))); + // two rewriters; second rewriter overwrites configs from first + fullConfig = new HashMap<>(baseConfigMap); + fullConfig.put(JobConfig.CONFIG_REWRITERS, REWRITER_NAME + "," + OTHER_REWRITER_NAME); + fullConfig.put(String.format(JobConfig.CONFIG_REWRITER_CLASS, REWRITER_NAME), NewPropertyRewriter.class.getName()); + fullConfig.put(String.format(JobConfig.CONFIG_REWRITER_CLASS, OTHER_REWRITER_NAME), + UpdatePropertyRewriter.class.getName()); + expectedConfigMap = new HashMap<>(fullConfig); + expectedConfigMap.put(NEW_CONFIG_KEY, CONFIG_VALUE + CONFIG_VALUE); + assertEquals(new MapConfig(expectedConfigMap), ConfigUtil.rewriteConfig(new MapConfig(fullConfig))); } - @Test - public void testRewriterWithConfigRewriter() { - configMap.put("job.config.rewriters", "testRewriter"); - configMap.put("job.config.rewriter.testRewriter.value", "rewrittenTest"); + /** + * This fails because Util will interpret the empty string value as a single rewriter which has the empty string as a + * name, and there is no rewriter class config for a rewriter name which is the empty string. + * TODO: should this be fixed to interpret the empty string as an empty list? + */ + @Test(expected = SamzaException.class) + public void testRewriteConfigConfigRewritersEmptyString() { + Config config = new MapConfig(ImmutableMap.of(JobConfig.CONFIG_REWRITERS, "")); + ConfigUtil.rewriteConfig(config); + } + + @Test(expected = SamzaException.class) + public void testRewriteConfigNoClassForConfigRewriterName() { + Config config = + new MapConfig(ImmutableMap.of(CONFIG_KEY, CONFIG_VALUE, JobConfig.CONFIG_REWRITERS, "unknownRewriter")); + ConfigUtil.rewriteConfig(config); + } - Config config = ConfigUtil.rewriteConfig(new MapConfig(configMap)); - assertEquals("rewrittenTest", config.get("value")); + @Test(expected = SamzaException.class) + public void testRewriteConfigRewriterClassDoesNotExist() { + Config config = new MapConfig(ImmutableMap.of(CONFIG_KEY, CONFIG_VALUE, JobConfig.CONFIG_REWRITERS, REWRITER_NAME, + String.format(JobConfig.CONFIG_REWRITER_CLASS, REWRITER_NAME), "not_a_class")); + ConfigUtil.rewriteConfig(config); } @Test - public void testGetRewriterWithoutConfigRewriter() { - Config config = ConfigUtil.rewriteConfig(new MapConfig(configMap)); - assertEquals(config, new MapConfig(configMap)); + public void testApplyRewriter() { + // new property + Map fullConfig = + ImmutableMap.of(CONFIG_KEY, CONFIG_VALUE, String.format(JobConfig.CONFIG_REWRITER_CLASS, REWRITER_NAME), + NewPropertyRewriter.class.getName()); + Map expectedConfigMap = new HashMap<>(fullConfig); + expectedConfigMap.put(NEW_CONFIG_KEY, CONFIG_VALUE); + assertEquals(new MapConfig(expectedConfigMap), ConfigUtil.applyRewriter(new MapConfig(fullConfig), REWRITER_NAME)); + + // update property + fullConfig = + ImmutableMap.of(CONFIG_KEY, CONFIG_VALUE, String.format(JobConfig.CONFIG_REWRITER_CLASS, REWRITER_NAME), + UpdatePropertyRewriter.class.getName()); + expectedConfigMap = new HashMap<>(fullConfig); + expectedConfigMap.put(CONFIG_KEY, CONFIG_VALUE + CONFIG_VALUE); + assertEquals(new MapConfig(expectedConfigMap), ConfigUtil.applyRewriter(new MapConfig(fullConfig), REWRITER_NAME)); + + // remove property + fullConfig = + ImmutableMap.of(CONFIG_KEY, CONFIG_VALUE, String.format(JobConfig.CONFIG_REWRITER_CLASS, REWRITER_NAME), + DeletePropertyRewriter.class.getName()); + expectedConfigMap = new HashMap<>(fullConfig); + expectedConfigMap.remove(CONFIG_KEY); + assertEquals(new MapConfig(expectedConfigMap), ConfigUtil.applyRewriter(new MapConfig(fullConfig), REWRITER_NAME)); } - @Test (expected = RuntimeException.class) - public void testGetRewriterWithExceptoion() { - configMap.put("job.config.rewriters", "testNoneRewriter"); - ConfigUtil.rewriteConfig(new MapConfig(configMap)); + @Test(expected = SamzaException.class) + public void testApplyRewriterNoClassForConfigRewriterName() { + Map fullConfig = ImmutableMap.of(CONFIG_KEY, CONFIG_VALUE); + ConfigUtil.applyRewriter(new MapConfig(fullConfig), REWRITER_NAME); + } + + @Test(expected = SamzaException.class) + public void testApplyRewriterClassDoesNotExist() { + Map fullConfig = + ImmutableMap.of(CONFIG_KEY, CONFIG_VALUE, String.format(JobConfig.CONFIG_REWRITER_CLASS, REWRITER_NAME), + "not_a_class"); + Config expectedConfig = new MapConfig(ImmutableMap.of(CONFIG_KEY, CONFIG_VALUE, NEW_CONFIG_KEY, CONFIG_VALUE)); + assertEquals(expectedConfig, ConfigUtil.applyRewriter(new MapConfig(fullConfig), REWRITER_NAME)); + } + + /** + * Adds a new config entry for the key {@link #NEW_CONFIG_KEY} which has the same value as {@link #CONFIG_KEY}. + */ + public static class NewPropertyRewriter implements ConfigRewriter { + @Override + public Config rewrite(String name, Config config) { + ImmutableMap.Builder newConfigMapBuilder = new ImmutableMap.Builder<>(); + newConfigMapBuilder.putAll(config); + newConfigMapBuilder.put(NEW_CONFIG_KEY, config.get(CONFIG_KEY)); + return new MapConfig(newConfigMapBuilder.build()); + } + } + + /** + * If an entry at {@link #NEW_CONFIG_KEY} exists, overwrites it to be the value concatenated with itself. Otherwise, + * updates the entry at {@link #CONFIG_KEY} to be the value concatenated to itself. + */ + public static class UpdatePropertyRewriter implements ConfigRewriter { + @Override + public Config rewrite(String name, Config config) { + Map newConfigMap = new HashMap<>(config); + if (config.containsKey(NEW_CONFIG_KEY)) { + // for testing overwriting of new configs + newConfigMap.put(NEW_CONFIG_KEY, config.get(NEW_CONFIG_KEY) + config.get(NEW_CONFIG_KEY)); + } else { + newConfigMap.put(CONFIG_KEY, config.get(CONFIG_KEY) + config.get(CONFIG_KEY)); + } + return new MapConfig(newConfigMap); + } } - public static class TestConfigRewriter implements ConfigRewriter { + /** + * Removes config entry for the key {@link #CONFIG_KEY} and {@link #NEW_CONFIG_KEY}. + */ + public static class DeletePropertyRewriter implements ConfigRewriter { @Override public Config rewrite(String name, Config config) { - Map configMap = new HashMap<>(config); - configMap.putAll(config.subset(String.format("job.config.rewriter.%s.", name))); - return new MapConfig(configMap); + Map newConfigMap = new HashMap<>(config); + newConfigMap.remove(CONFIG_KEY); + return new MapConfig(newConfigMap); } } } diff --git a/samza-core/src/test/java/org/apache/samza/util/TestUtil.java b/samza-core/src/test/java/org/apache/samza/util/TestUtil.java new file mode 100644 index 0000000000..2eb2e89833 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/util/TestUtil.java @@ -0,0 +1,201 @@ +/* + * 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.util; + +import java.net.Inet4Address; +import java.net.InetAddress; +import java.net.NetworkInterface; +import java.net.UnknownHostException; +import java.util.Arrays; +import java.util.Collections; +import com.google.common.collect.ImmutableMap; +import org.apache.samza.config.ApplicationConfig; +import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; +import org.apache.samza.config.TaskConfig; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.junit.Assert.assertEquals; +import static org.mockito.AdditionalMatchers.aryEq; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.mockStatic; +import static org.powermock.api.mockito.PowerMockito.when; + + +@RunWith(PowerMockRunner.class) +@PrepareForTest(Util.class) // need this to be able to use powermock with system classes like InetAddress +public class TestUtil { + @Test + public void testEnvVarEscape() { + // no special characters in original + String noSpecialCharacters = "hello world 123 .?!"; + assertEquals(noSpecialCharacters, Util.envVarEscape(noSpecialCharacters)); + + String withSpecialCharacters = "quotation \" apostrophe '"; + String escaped = "quotation \\\" apostrophe \\'"; + assertEquals(escaped, Util.envVarEscape(withSpecialCharacters)); + } + + /** + * It's difficult to explicitly test having an actual version and using the fallback, due to the usage of methods of + * Class. + */ + @Test + public void testGetSamzaVersion() { + String utilImplementationVersion = Util.class.getPackage().getImplementationVersion(); + String expectedVersion = + (utilImplementationVersion != null) ? utilImplementationVersion : Util.FALLBACK_VERSION; + assertEquals(expectedVersion, Util.getSamzaVersion()); + } + + /** + * It's difficult to explicitly test having an actual version and using the fallback, due to the usage of methods of + * Class. + */ + @Test + public void testGetTaskClassVersion() { + // cannot find app nor task + assertEquals(Util.FALLBACK_VERSION, Util.getTaskClassVersion(new MapConfig())); + + // only app + String appClassVersion = MyAppClass.class.getPackage().getImplementationVersion(); + String expectedAppClassVersion = (appClassVersion != null) ? appClassVersion : Util.FALLBACK_VERSION; + Config config = new MapConfig(ImmutableMap.of(ApplicationConfig.APP_CLASS, MyAppClass.class.getName())); + assertEquals(expectedAppClassVersion, Util.getTaskClassVersion(config)); + + // only task + String taskClassVersion = MyTaskClass.class.getPackage().getImplementationVersion(); + String expectedTaskClassVersion = (taskClassVersion != null) ? taskClassVersion : Util.FALLBACK_VERSION; + config = new MapConfig(ImmutableMap.of(TaskConfig.TASK_CLASS, MyTaskClass.class.getName())); + assertEquals(expectedTaskClassVersion, Util.getTaskClassVersion(config)); + + // both app and task; choose app + config = new MapConfig(ImmutableMap.of(ApplicationConfig.APP_CLASS, MyAppClass.class.getName(), + // shouldn't even try to load the task class + TaskConfig.TASK_CLASS, "this_is_not_a_class")); + assertEquals(expectedAppClassVersion, Util.getTaskClassVersion(config)); + } + + @Test + public void testGetLocalHostNotLoopbackAddress() throws UnknownHostException { + mockStatic(InetAddress.class); + InetAddress inetAddressLocalHost = mock(InetAddress.class); + when(inetAddressLocalHost.isLoopbackAddress()).thenReturn(false); + when(InetAddress.getLocalHost()).thenReturn(inetAddressLocalHost); + assertEquals(inetAddressLocalHost, Util.getLocalHost()); + } + + @Test + public void testGetLocalHostLoopbackAddressNoExternalAddressFound() throws Exception { + mockStatic(InetAddress.class, NetworkInterface.class); + InetAddress inetAddressLocalHost = mock(InetAddress.class); + when(inetAddressLocalHost.isLoopbackAddress()).thenReturn(true); + when(InetAddress.getLocalHost()).thenReturn(inetAddressLocalHost); + + // network interfaces return addresses which are not external + InetAddress linkLocalAddress = mock(InetAddress.class); + when(linkLocalAddress.isLinkLocalAddress()).thenReturn(true); + InetAddress loopbackAddress = mock(InetAddress.class); + when(loopbackAddress.isLinkLocalAddress()).thenReturn(false); + when(loopbackAddress.isLoopbackAddress()).thenReturn(true); + NetworkInterface networkInterface0 = mock(NetworkInterface.class); + when(networkInterface0.getInetAddresses()).thenReturn( + Collections.enumeration(Arrays.asList(linkLocalAddress, loopbackAddress))); + NetworkInterface networkInterface1 = mock(NetworkInterface.class); + when(networkInterface1.getInetAddresses()).thenReturn( + Collections.enumeration(Collections.singletonList(loopbackAddress))); + when(NetworkInterface.getNetworkInterfaces()).thenReturn( + Collections.enumeration(Arrays.asList(networkInterface0, networkInterface1))); + + assertEquals(inetAddressLocalHost, Util.getLocalHost()); + } + + @Test + public void testGetLocalHostExternalInet4Address() throws Exception { + mockStatic(InetAddress.class, NetworkInterface.class); + InetAddress inetAddressLocalHost = mock(InetAddress.class); + when(inetAddressLocalHost.isLoopbackAddress()).thenReturn(true); + when(InetAddress.getLocalHost()).thenReturn(inetAddressLocalHost); + + InetAddress linkLocalAddress = mock(InetAddress.class); + when(linkLocalAddress.isLinkLocalAddress()).thenReturn(true); + Inet4Address externalInet4Address = mock(Inet4Address.class); + when(externalInet4Address.isLinkLocalAddress()).thenReturn(false); + when(externalInet4Address.isLoopbackAddress()).thenReturn(false); + byte[] externalInet4AddressBytes = new byte[]{0, 1, 2, 3}; + when(externalInet4Address.getAddress()).thenReturn(externalInet4AddressBytes); + InetAddress otherExternalAddress = mock(InetAddress.class); // not Inet4Address + when(otherExternalAddress.isLinkLocalAddress()).thenReturn(false); + when(otherExternalAddress.isLoopbackAddress()).thenReturn(false); + + NetworkInterface networkInterfaceLinkLocal = mock(NetworkInterface.class); + when(networkInterfaceLinkLocal.getInetAddresses()).thenReturn( + Collections.enumeration(Collections.singletonList(linkLocalAddress))); + NetworkInterface networkInterfaceExternal = mock(NetworkInterface.class); + when(networkInterfaceExternal.getInetAddresses()).thenReturn( + Collections.enumeration(Arrays.asList(otherExternalAddress, externalInet4Address))); + when(NetworkInterface.getNetworkInterfaces()).thenReturn( + Collections.enumeration(Arrays.asList(networkInterfaceLinkLocal, networkInterfaceExternal))); + + InetAddress finalInetAddress = mock(InetAddress.class); + when(InetAddress.getByAddress(aryEq(externalInet4AddressBytes))).thenReturn(finalInetAddress); + + assertEquals(finalInetAddress, Util.getLocalHost()); + } + + @Test + public void testGetLocalHostExternalAddressNotInet4Address() throws Exception { + mockStatic(InetAddress.class, NetworkInterface.class); + InetAddress inetAddressLocalHost = mock(InetAddress.class); + when(inetAddressLocalHost.isLoopbackAddress()).thenReturn(true); + when(InetAddress.getLocalHost()).thenReturn(inetAddressLocalHost); + + byte[] externalAddressBytes = new byte[]{0, 1, 2, 3, 4, 5}; + InetAddress externalAddress = mock(InetAddress.class); + when(externalAddress.isLinkLocalAddress()).thenReturn(false); + when(externalAddress.isLoopbackAddress()).thenReturn(false); + when(externalAddress.getAddress()).thenReturn(externalAddressBytes); + + NetworkInterface networkInterface = mock(NetworkInterface.class); + when(networkInterface.getInetAddresses()).thenReturn( + Collections.enumeration(Collections.singletonList(externalAddress))); + when(NetworkInterface.getNetworkInterfaces()).thenReturn( + Collections.enumeration(Collections.singletonList(networkInterface))); + + InetAddress finalInetAddress = mock(InetAddress.class); + when(InetAddress.getByAddress(aryEq(externalAddressBytes))).thenReturn(finalInetAddress); + + assertEquals(finalInetAddress, Util.getLocalHost()); + } + + /** + * No requirement for this test that this extends any other class. Just need some placeholder class. + */ + public static class MyAppClass { + } + + /** + * No requirement for this test that this extends any other class. Just need some placeholder class. + */ + public static class MyTaskClass { + } +} diff --git a/samza-core/src/test/scala/org/apache/samza/util/TestUtil.scala b/samza-core/src/test/scala/org/apache/samza/util/TestUtil.scala deleted file mode 100644 index ba3b5df483..0000000000 --- a/samza-core/src/test/scala/org/apache/samza/util/TestUtil.scala +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.util - -import org.junit.Assert._ -import org.junit.Test -import org.apache.samza.config.MapConfig - -class TestUtil { - @Test - def testGetLocalHost(): Unit = { - assertNotNull(Util.getLocalHost) - } - - @Test - def testGetObjExistingClass() { - val obj = Util.getObj("org.apache.samza.config.MapConfig", classOf[MapConfig]) - assertNotNull(obj) - assertEquals(classOf[MapConfig], obj.getClass()) - } - - @Test(expected = classOf[ClassNotFoundException]) - def testGetObjNonexistentClass() { - Util.getObj("this.class.does.NotExist", classOf[Object]) - assert(false, "This should not get hit.") - } -} diff --git a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala index b20704a61e..706ae65d34 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala @@ -26,7 +26,7 @@ import org.apache.samza.metrics.MetricsRegistry import org.apache.samza.system.{StreamSpec, SystemFactory} import org.apache.samza.system.kafka.KafkaStreamSpec import org.apache.samza.util.ScalaJavaUtil.JavaOptionals -import org.apache.samza.util.{KafkaUtil, Logging, Util, _} +import org.apache.samza.util.{KafkaUtil, Logging, _} class KafkaCheckpointManagerFactory extends CheckpointManagerFactory with Logging { @@ -45,7 +45,7 @@ class KafkaCheckpointManagerFactory extends CheckpointManagerFactory with Loggin .toOption .getOrElse(throw new SamzaException("Missing configuration: " + SystemConfig.SYSTEM_FACTORY_FORMAT format checkpointSystemName)) - val checkpointSystemFactory = Util.getObj(checkpointSystemFactoryName, classOf[SystemFactory]) + val checkpointSystemFactory = ReflectionUtil.getObj(checkpointSystemFactoryName, classOf[SystemFactory]) val checkpointTopic = KafkaUtil.getCheckpointTopic(jobName, jobId, config) info(s"Creating a KafkaCheckpointManager to consume from $checkpointTopic") diff --git a/samza-yarn/src/main/java/org/apache/samza/validation/YarnJobValidationTool.java b/samza-yarn/src/main/java/org/apache/samza/validation/YarnJobValidationTool.java index 376e113051..bbcc976386 100644 --- a/samza-yarn/src/main/java/org/apache/samza/validation/YarnJobValidationTool.java +++ b/samza-yarn/src/main/java/org/apache/samza/validation/YarnJobValidationTool.java @@ -33,25 +33,21 @@ import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.samza.SamzaException; -import org.apache.samza.clustermanager.ClusterBasedJobCoordinator; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; -import org.apache.samza.config.MapConfig; import org.apache.samza.coordinator.JobModelManager; import org.apache.samza.coordinator.metadatastore.CoordinatorStreamStore; -import org.apache.samza.coordinator.stream.CoordinatorStreamManager; import org.apache.samza.coordinator.stream.messages.SetContainerHostMapping; -import org.apache.samza.job.model.JobModel; import org.apache.samza.job.yarn.ClientHelper; import org.apache.samza.metrics.JmxMetricsAccessor; import org.apache.samza.metrics.MetricsRegistry; import org.apache.samza.metrics.MetricsRegistryMap; import org.apache.samza.metrics.MetricsValidator; import org.apache.samza.storage.ChangelogStreamManager; +import org.apache.samza.util.CommandLine; import org.apache.samza.util.CoordinatorStreamUtil; -import org.apache.samza.util.Util; +import org.apache.samza.util.ReflectionUtil; import org.apache.samza.util.hadoop.HttpFileSystem; -import org.apache.samza.util.CommandLine; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -196,7 +192,7 @@ public static void main(String [] args) throws Exception { MetricsValidator validator = null; if (options.has(validatorOpt)) { String validatorClass = options.valueOf(validatorOpt); - validator = Util.getObj(validatorClass, MetricsValidator.class); + validator = ReflectionUtil.getObj(validatorClass, MetricsValidator.class); } YarnConfiguration hadoopConfig = new YarnConfiguration(); diff --git a/samza-yarn/src/main/resources/scalate/WEB-INF/views/index.scaml b/samza-yarn/src/main/resources/scalate/WEB-INF/views/index.scaml index 5c31ccacb6..7fe6305138 100644 --- a/samza-yarn/src/main/resources/scalate/WEB-INF/views/index.scaml +++ b/samza-yarn/src/main/resources/scalate/WEB-INF/views/index.scaml @@ -25,7 +25,7 @@ -@ val appMasterClasspath: String = scala.util.Properties.javaClassPath -@ val javaVmVersion: String = scala.util.Properties.javaVmVersion -@ val javaVmName: String = scala.util.Properties.javaVmName --@ val samzaVersion: String = org.apache.samza.util.Util.getClass.getPackage.getImplementationVersion +-@ val samzaVersion: String = classOf[org.apache.samza.util.Util].getPackage.getImplementationVersion - attributes("title") = jobName %div.col-xs-2.menu From 5e9244ac822ff4ca885e3878a73157e874a99a36 Mon Sep 17 00:00:00 2001 From: Hai Lu Date: Thu, 5 Dec 2019 11:27:57 -0800 Subject: [PATCH 13/49] update gradle properties to mark 1.4.0 as the next version (#1230) --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index f917f87d2b..d3ce6118e2 100644 --- a/gradle.properties +++ b/gradle.properties @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. group=org.apache.samza -version=1.3.0-SNAPSHOT +version=1.4.0-SNAPSHOT scalaSuffix=2.11 gradleVersion=5.2.1 From 6d206d5c45425828d89e48a9cb5b7404d89eb12c Mon Sep 17 00:00:00 2001 From: lakshmi-manasa-g Date: Fri, 6 Dec 2019 14:18:34 -0800 Subject: [PATCH 14/49] [DOCS] Update kafka consumer command to use bootstrap-server (#1232) --- docs/startup/hello-samza/versioned/index.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/startup/hello-samza/versioned/index.md b/docs/startup/hello-samza/versioned/index.md index 0b0ae9343c..325334ba27 100644 --- a/docs/startup/hello-samza/versioned/index.md +++ b/docs/startup/hello-samza/versioned/index.md @@ -75,7 +75,7 @@ deploy/samza/bin/run-app.sh --config-factory=org.apache.samza.config.factories.P The job will consume a feed of real-time edits from Wikipedia, and produce them to a Kafka topic called "wikipedia-raw". Give the job a minute to startup, and then tail the Kafka topic: {% highlight bash %} -deploy/kafka/bin/kafka-console-consumer.sh --zookeeper localhost:2181 --topic wikipedia-raw +deploy/kafka/bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic wikipedia-raw {% endhighlight %} Pretty neat, right? Now, check out the YARN UI again ([http://localhost:8088](http://localhost:8088)). This time around, you'll see your Samza job is running! @@ -94,13 +94,13 @@ deploy/samza/bin/run-app.sh --config-factory=org.apache.samza.config.factories.P The first job (wikipedia-parser) parses the messages in wikipedia-raw, and extracts information about the size of the edit, who made the change, etc. You can take a look at its output with: {% highlight bash %} -deploy/kafka/bin/kafka-console-consumer.sh --zookeeper localhost:2181 --topic wikipedia-edits +deploy/kafka/bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic wikipedia-edits {% endhighlight %} The last job (wikipedia-stats) reads messages from the wikipedia-edits topic, and calculates counts, every ten seconds, for all edits that were made during that window. It outputs these counts to the wikipedia-stats topic. {% highlight bash %} -deploy/kafka/bin/kafka-console-consumer.sh --zookeeper localhost:2181 --topic wikipedia-stats +deploy/kafka/bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic wikipedia-stats {% endhighlight %} The messages in the stats topic look like this: From c8417b78105b75c4eba38487c9e14a45c6974099 Mon Sep 17 00:00:00 2001 From: Hai Lu Date: Sun, 8 Dec 2019 21:50:15 -0800 Subject: [PATCH 15/49] doc and blog related update on master branch for 1.3.0 release (#1231) doc and blog related update on master branch for 1.3.0 release --- ...cing-the-release-of-apache-samza--1.3.0.md | 102 ++++++++++++++++++ docs/_config.yml | 2 +- docs/_releases/1.3.0.md | 93 ++++++++++++++++ docs/archive/index.html | 8 ++ .../versioned/hello-samza-high-level-yarn.md | 2 +- .../versioned/hello-samza-high-level-zk.md | 2 +- .../versioned/samza-rest-getting-started.md | 2 +- docs/startup/download/index.md | 18 ++-- docs/startup/hello-samza/versioned/index.md | 2 +- 9 files changed, 218 insertions(+), 13 deletions(-) create mode 100644 docs/_blog/2019-12-09-announcing-the-release-of-apache-samza--1.3.0.md create mode 100644 docs/_releases/1.3.0.md diff --git a/docs/_blog/2019-12-09-announcing-the-release-of-apache-samza--1.3.0.md b/docs/_blog/2019-12-09-announcing-the-release-of-apache-samza--1.3.0.md new file mode 100644 index 0000000000..7af7c2aeeb --- /dev/null +++ b/docs/_blog/2019-12-09-announcing-the-release-of-apache-samza--1.3.0.md @@ -0,0 +1,102 @@ +--- +layout: blog +title: Announcing the release of Apache Samza 1.3.0 +icon: git-pull-request +authors: + - name: Hai Lu + website: + image: +excerpt_separator: +--- + + + +# **Announcing the release of Apache Samza 1.3.0** + + + + +**IMPORTANT NOTE**: We may introduce a **backward incompatible changes regarding samza job submission** in the future 1.4 release. Details can be found on [SEP-23: Simplify Job Runner](https://cwiki.apache.org/confluence/display/SAMZA/SEP-23%3A+Simplify+Job+Runner) + +We’re thrilled to announce the release of Apache Samza 1.3.0. + +Today Samza forms the backbone of hundreds of real-time production applications across a multitude of companies, such as LinkedIn, VMWare, Slack, Redfin among many others. Samza provides leading support for large-scale stateful stream processing with: + +* First class support for local state (with RocksDB store). This allows a stateful application to scale up to 1.1 Million events/sec on a single machine with SSD. + +* Support for incremental checkpointing of state instead of full snapshots. This enables Samza to scale to applications with very large state. + +* A fully asynchronous programming model that makes parallelizing remote calls efficient and effortless. + +* High level API for expressing complex stream processing pipelines in a few lines of code. + +* Beam Samza Runner that marries Beam’s best in class support for EventTime based windowed processing and sophisticated triggering with Samza’s stable and scalable stateful processing model. + +* A fully pluggable model for input sources (e.g. Kafka, Kinesis, DynamoDB streams etc.) and output systems (HDFS, Kafka, ElastiCache etc.). + +* A Table API that provides a common abstraction for accessing remote or local databases and allowing developers are able to "join" an input event stream with such a Table. + +* Flexible deployment model for running the applications in any hosting environment and with cluster managers other than YARN. + +* Features like canaries, upgrades and rollbacks that support extremely large deployments with minimal downtime. + +### **New Features, Upgrades and Bug Fixes:** +This release brings the following features, upgrades, and capabilities (highlights): + +* Startpoint support improvement + +* Samza SQL improvement + +* Table API improvement + +* Miscellaneous bug fixes + +Full list of the jiras addressed in this release can be found [here](https://issues.apache.org/jira/browse/SAMZA-2354?jql=project%20%3D%20%22SAMZA%22%20and%20fixVersion%20in%20(1.3)). +### **Upgrading your application to Apache Samza 1.3.0** + +### Startpoint support improvement +[SAMZA-2201](https://issues.apache.org/jira/browse/SAMZA-2201) Startpoints - Integrate fan out with job coordinators + +[SAMZA-2215](https://issues.apache.org/jira/browse/SAMZA-2215) StartpointManager fix for previous CoordinatorStreamStore refactor + +[SAMZA-2220](https://issues.apache.org/jira/browse/SAMZA-2220) Startpoints - Fully encapsulate resolution of starting offsets in OffsetManager + +### Samza SQL improvement +[SAMZA-2234](https://issues.apache.org/jira/browse/SAMZA-2234) Samza SQL : Provide access to Samza context to the Sama SQL UDFs + +[SAMZA-2313](https://issues.apache.org/jira/browse/SAMZA-2313) Samza-sql: Add validation for Samza sql statements + +[SAMZA-2354](https://issues.apache.org/jira/browse/SAMZA-2354) Improve UDF discovery in samza-sql + +#### Table API improvement +[SAMZA-2191](https://issues.apache.org/jira/browse/SAMZA-2191) support batching for remote tables + +[SAMZA-2200](https://issues.apache.org/jira/browse/SAMZA-2200) update table sendTo() and join() operation to accept additional arguments + +[SAMZA-2219](https://issues.apache.org/jira/browse/SAMZA-2219) Add a dummy table read function + +[SAMZA-2309](https://issues.apache.org/jira/browse/SAMZA-2309) Remote table descriptor requires read function + +#### Miscellaneous bug fixing +[SAMZA-2198](https://issues.apache.org/jira/browse/SAMZA-2198) containers process always takes task.shutdown.ms to shut down + +[SAMZA-2293](https://issues.apache.org/jira/browse/SAMZA-2293) Propagate the watermark future to StreamOperatorTask correctly + +### Sources downloads +A source download of Samza 1.3.0 is available [here](https://dist.apache.org/repos/dist/release/samza/1.3.0/), and is also available in Apache’s Maven repository. See Samza’s download [page](https://samza.apache.org/startup/download/) for details and Samza’s feature preview for new features. + diff --git a/docs/_config.yml b/docs/_config.yml index 31edfbcda5..695e19217c 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -25,7 +25,7 @@ exclude: [_docs] baseurl: http://samza.apache.org version: latest # this is the version you will go if you click 'switch version' in "latest" pages. -latest-release: '1.2.0' +latest-release: '1.3.0' collections: menu: output: false diff --git a/docs/_releases/1.3.0.md b/docs/_releases/1.3.0.md new file mode 100644 index 0000000000..5d7fb80bb9 --- /dev/null +++ b/docs/_releases/1.3.0.md @@ -0,0 +1,93 @@ +--- +version: '1.3' +order: 130 +layout: page +menu_title: '1.3' +title: Apache Samza 1.3 [Docs] +--- + + +**IMPORTANT NOTE**: We may introduce a **backward incompatible changes regarding samza job submission** in the future 1.4 release. Details can be found on [SEP-23: Simplify Job Runner](https://cwiki.apache.org/confluence/display/SAMZA/SEP-23%3A+Simplify+Job+Runner) + +We’re thrilled to announce the release of Apache Samza 1.3.0. + +Today Samza forms the backbone of hundreds of real-time production applications across a multitude of companies, such as LinkedIn, VMWare, Slack, Redfin among many others. Samza provides leading support for large-scale stateful stream processing with: + +* First class support for local state (with RocksDB store). This allows a stateful application to scale up to 1.1 Million events/sec on a single machine with SSD. + +* Support for incremental checkpointing of state instead of full snapshots. This enables Samza to scale to applications with very large state. + +* A fully asynchronous programming model that makes parallelizing remote calls efficient and effortless. + +* High level API for expressing complex stream processing pipelines in a few lines of code. + +* Beam Samza Runner that marries Beam’s best in class support for EventTime based windowed processing and sophisticated triggering with Samza’s stable and scalable stateful processing model. + +* A fully pluggable model for input sources (e.g. Kafka, Kinesis, DynamoDB streams etc.) and output systems (HDFS, Kafka, ElastiCache etc.). + +* A Table API that provides a common abstraction for accessing remote or local databases and allowing developers are able to "join" an input event stream with such a Table. + +* Flexible deployment model for running the applications in any hosting environment and with cluster managers other than YARN. + +* Features like canaries, upgrades and rollbacks that support extremely large deployments with minimal downtime. + +### **New Features, Upgrades and Bug Fixes:** +This release brings the following features, upgrades, and capabilities (highlights): + +* Startpoint support improvement + +* Samza SQL improvement + +* Table API improvement + +* Miscellaneous bug fixes + +Full list of the jiras addressed in this release can be found [here](https://issues.apache.org/jira/browse/SAMZA-2354?jql=project%20%3D%20%22SAMZA%22%20and%20fixVersion%20in%20(1.3)). +### **Upgrading your application to Apache Samza 1.3.0** + +### Startpoint support improvement +[SAMZA-2201](https://issues.apache.org/jira/browse/SAMZA-2201) Startpoints - Integrate fan out with job coordinators + +[SAMZA-2215](https://issues.apache.org/jira/browse/SAMZA-2215) StartpointManager fix for previous CoordinatorStreamStore refactor + +[SAMZA-2220](https://issues.apache.org/jira/browse/SAMZA-2220) Startpoints - Fully encapsulate resolution of starting offsets in OffsetManager + +### Samza SQL improvement +[SAMZA-2234](https://issues.apache.org/jira/browse/SAMZA-2234) Samza SQL : Provide access to Samza context to the Sama SQL UDFs + +[SAMZA-2313](https://issues.apache.org/jira/browse/SAMZA-2313) Samza-sql: Add validation for Samza sql statements + +[SAMZA-2354](https://issues.apache.org/jira/browse/SAMZA-2354) Improve UDF discovery in samza-sql + +#### Table API improvement +[SAMZA-2191](https://issues.apache.org/jira/browse/SAMZA-2191) support batching for remote tables + +[SAMZA-2200](https://issues.apache.org/jira/browse/SAMZA-2200) update table sendTo() and join() operation to accept additional arguments + +[SAMZA-2219](https://issues.apache.org/jira/browse/SAMZA-2219) Add a dummy table read function + +[SAMZA-2309](https://issues.apache.org/jira/browse/SAMZA-2309) Remote table descriptor requires read function + +#### Miscellaneous bug fixing +[SAMZA-2198](https://issues.apache.org/jira/browse/SAMZA-2198) containers process always takes task.shutdown.ms to shut down + +[SAMZA-2293](https://issues.apache.org/jira/browse/SAMZA-2293) Propagate the watermark future to StreamOperatorTask correctly + +### Sources downloads +A source download of Samza 1.3.0 is available [here](https://dist.apache.org/repos/dist/release/samza/1.3.0/), and is also available in Apache’s Maven repository. See Samza’s download [page](https://samza.apache.org/startup/download/) for details and Samza’s feature preview for new features. + diff --git a/docs/archive/index.html b/docs/archive/index.html index bddc271b88..d83937fa50 100644 --- a/docs/archive/index.html +++ b/docs/archive/index.html @@ -27,6 +27,14 @@

Latest Release

  • Hello Samza
  • +

    1.3 Release

    + + +

    1.2 Release

      diff --git a/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md b/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md index d3796ee081..69abf68f65 100644 --- a/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md +++ b/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md @@ -63,7 +63,7 @@ Then, you can continue w/ the following command in hello-samza project: {% highlight bash %} mvn clean package mkdir -p deploy/samza -tar -xvf ./target/hello-samza-1.2.0-SNAPSHOT-dist.tar.gz -C deploy/samza +tar -xvf ./target/hello-samza-1.4.0-SNAPSHOT-dist.tar.gz -C deploy/samza {% endhighlight %} ### Run a Samza Application diff --git a/docs/learn/tutorials/versioned/hello-samza-high-level-zk.md b/docs/learn/tutorials/versioned/hello-samza-high-level-zk.md index 1dfefed5b2..aa139e0674 100644 --- a/docs/learn/tutorials/versioned/hello-samza-high-level-zk.md +++ b/docs/learn/tutorials/versioned/hello-samza-high-level-zk.md @@ -59,7 +59,7 @@ With the environment setup complete, let us move on to building the hello-samza {% highlight bash %} mvn clean package mkdir -p deploy/samza -tar -xvf ./target/hello-samza-1.3.0-SNAPSHOT-dist.tar.gz -C deploy/samza +tar -xvf ./target/hello-samza-1.4.0-SNAPSHOT-dist.tar.gz -C deploy/samza {% endhighlight %} We are now all set to deploy the application locally. diff --git a/docs/learn/tutorials/versioned/samza-rest-getting-started.md b/docs/learn/tutorials/versioned/samza-rest-getting-started.md index 1a64dd2565..bd8137aa9e 100644 --- a/docs/learn/tutorials/versioned/samza-rest-getting-started.md +++ b/docs/learn/tutorials/versioned/samza-rest-getting-started.md @@ -48,7 +48,7 @@ Run the following commands: {% highlight bash %} cd samza-rest/build/distributions/ mkdir -p deploy/samza-rest -tar -xvf ./samza-rest-1.3.0-SNAPSHOT.tgz -C deploy/samza-rest +tar -xvf ./samza-rest-1.4.0-SNAPSHOT.tgz -C deploy/samza-rest {% endhighlight %} #### Configure the Installations Path diff --git a/docs/startup/download/index.md b/docs/startup/download/index.md index 3f48ab1cda..bc29fff4ff 100644 --- a/docs/startup/download/index.md +++ b/docs/startup/download/index.md @@ -31,6 +31,7 @@ Starting from 2016, Samza will begin requiring JDK8 or higher. Please see [this Samza tools package contains command line tools that user can run to use Samza and it's input/output systems. + * [samza-tools_2.11-1.3.0.tgz](http://www-us.apache.org/dist/samza/1.3.0/samza-tools_2.11-1.3.0.tgz) * [samza-tools_2.11-1.2.0.tgz](http://www-us.apache.org/dist/samza/1.2.0/samza-tools_2.11-1.2.0.tgz) * [samza-tools_2.11-1.1.0.tgz](http://www-us.apache.org/dist/samza/1.1.0/samza-tools_2.11-1.1.0.tgz) * [samza-tools-1.0.0.tgz](http://www-us.apache.org/dist/samza/1.0.0/samza-tools-1.0.0.tgz) @@ -38,6 +39,7 @@ Starting from 2016, Samza will begin requiring JDK8 or higher. Please see [this ### Source Releases + * [samza-sources-1.3.0.tgz](http://www.apache.org/dyn/closer.lua/samza/1.3.0) * [samza-sources-1.2.0.tgz](http://www.apache.org/dyn/closer.lua/samza/1.2.0) * [samza-sources-1.1.0.tgz](http://www.apache.org/dyn/closer.lua/samza/1.1.0) * [samza-sources-1.0.0.tgz](http://www.apache.org/dyn/closer.lua/samza/1.0.0) @@ -67,12 +69,12 @@ A Maven-based Samza project can pull in all required dependencies Samza dependen org.apache.samza samza-api - 1.2.0 + 1.3.0 org.apache.samza samza-core_2.11 - 1.2.0 + 1.3.0 runtime @@ -80,37 +82,37 @@ A Maven-based Samza project can pull in all required dependencies Samza dependen samza-shell dist tgz - 1.2.0 + 1.3.0 runtime org.apache.samza samza-yarn_2.11 - 1.2.0 + 1.3.0 runtime org.apache.samza samza-kv_2.11 - 1.2.0 + 1.3.0 runtime org.apache.samza samza-kv-rocksdb_2.11 - 1.2.0 + 1.3.0 runtime org.apache.samza samza-kv-inmemory_2.11 - 1.2.0 + 1.3.0 runtime org.apache.samza samza-kafka_2.11 - 1.2.0 + 1.3.0 runtime {% endhighlight %} diff --git a/docs/startup/hello-samza/versioned/index.md b/docs/startup/hello-samza/versioned/index.md index 325334ba27..58c5a4c4d4 100644 --- a/docs/startup/hello-samza/versioned/index.md +++ b/docs/startup/hello-samza/versioned/index.md @@ -61,7 +61,7 @@ Then, you can continue w/ the following command in hello-samza project: {% highlight bash %} mvn clean package mkdir -p deploy/samza -tar -xvf ./target/hello-samza-1.3.0-SNAPSHOT-dist.tar.gz -C deploy/samza +tar -xvf ./target/hello-samza-1.4.0-SNAPSHOT-dist.tar.gz -C deploy/samza {% endhighlight %} ### Run a Samza Job From c55e558ebc894905d3a4dcaf1a9f8d1d97f1dfdb Mon Sep 17 00:00:00 2001 From: Aditya Toomula Date: Mon, 9 Dec 2019 09:50:29 -0800 Subject: [PATCH 16/49] Fix AvroRelConverter to only consider cached schema while populating SamzaSqlRelRecord for all the nested records. --- .../org/apache/samza/sql/avro/AvroRelConverter.java | 13 +++---------- .../apache/samza/sql/interfaces/SqlIOConfig.java | 6 ++---- 2 files changed, 5 insertions(+), 14 deletions(-) diff --git a/samza-sql/src/main/java/org/apache/samza/sql/avro/AvroRelConverter.java b/samza-sql/src/main/java/org/apache/samza/sql/avro/AvroRelConverter.java index 026638477b..d70497e03a 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/avro/AvroRelConverter.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/avro/AvroRelConverter.java @@ -125,18 +125,11 @@ public static void fetchFieldNamesAndValuesFromIndexedRecord(IndexedRecord recor .collect(Collectors.toList())); } - private static SamzaSqlRelRecord convertToRelRecord(IndexedRecord avroRecord) { + private static SamzaSqlRelRecord convertToRelRecord(IndexedRecord avroRecord, Schema schema) { List fieldValues = new ArrayList<>(); List fieldNames = new ArrayList<>(); if (avroRecord != null) { - fieldNames.addAll( - avroRecord.getSchema().getFields().stream().map(Schema.Field::name).collect(Collectors.toList())); - fieldValues.addAll(avroRecord.getSchema() - .getFields() - .stream() - .map(f -> convertToJavaObject(avroRecord.get(avroRecord.getSchema().getField(f.name()).pos()), - getNonNullUnionSchema(avroRecord.getSchema().getField(f.name()).schema()))) - .collect(Collectors.toList())); + fetchFieldNamesAndValuesFromIndexedRecord(avroRecord, fieldNames, fieldValues, schema); } else { String msg = "Avro Record is null"; LOG.error(msg); @@ -231,7 +224,7 @@ public static Object convertToJavaObject(Object avroObj, Schema schema) { } switch (schema.getType()) { case RECORD: - return convertToRelRecord((IndexedRecord) avroObj); + return convertToRelRecord((IndexedRecord) avroObj, schema); case ARRAY: { ArrayList retVal = new ArrayList<>(); List avroArray; diff --git a/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SqlIOConfig.java b/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SqlIOConfig.java index 69d301d6c7..4350889b3b 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SqlIOConfig.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SqlIOConfig.java @@ -81,13 +81,11 @@ public SqlIOConfig(String systemName, String streamName, List sourcePart this.streamId = String.format("%s-%s", systemName, streamName); samzaRelConverterName = streamConfigs.get(CFG_SAMZA_REL_CONVERTER); - Validate.notEmpty(samzaRelConverterName, String.format("System %s is unknown. %s is not set or empty for this" - + " system", systemName, CFG_SAMZA_REL_CONVERTER)); + Validate.notEmpty(samzaRelConverterName, String.format("System %s is not supported. Please check if the system name is provided correctly.", systemName)); if (isRemoteTable()) { samzaRelTableKeyConverterName = streamConfigs.get(CFG_SAMZA_REL_TABLE_KEY_CONVERTER); - Validate.notEmpty(samzaRelTableKeyConverterName, String.format("System %s is unknown. %s is not set or empty for" - + " this system", systemName, CFG_SAMZA_REL_CONVERTER)); + Validate.notEmpty(samzaRelTableKeyConverterName, String.format("System %s is not supported. Please check if the system name is provided correctly.", systemName)); } else { samzaRelTableKeyConverterName = ""; } From d181ed894661cc68dd7a351f543743ad74a8270a Mon Sep 17 00:00:00 2001 From: mynameborat Date: Tue, 10 Dec 2019 14:04:45 -0800 Subject: [PATCH 17/49] SAMZA-2305: Stream processor should ensure previous container is stopped during a rebalance (#1213) Stream processor should ensure previous container is stopped during a rebalance --- .../apache/samza/storage/StorageEngine.java | 6 +- .../samza/processor/StreamProcessor.java | 72 ++++++++++++++----- ...nTransactionalStateTaskRestoreManager.java | 2 +- .../apache/samza/storage/StorageRecovery.java | 9 ++- .../samza/storage/TaskRestoreManager.java | 11 ++- .../TransactionalStateTaskRestoreManager.java | 2 +- .../samza/container/SamzaContainer.scala | 22 +++++- .../storage/ContainerStorageManager.java | 60 +++++++++++----- .../apache/samza/system/SystemConsumers.scala | 21 +++++- .../apache/samza/system/SystemProducers.scala | 17 ++++- .../samza/processor/TestStreamProcessor.java | 62 +++++++++++++++- .../samza/container/TestSamzaContainer.scala | 42 +++++++++++ .../storage/TestContainerStorageManager.java | 4 +- .../storage/kv/KeyValueStorageEngine.scala | 7 +- .../kv/TestKeyValueStorageEngine.scala | 48 +++++++++++++ 15 files changed, 332 insertions(+), 53 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java b/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java index 8add1de570..7b12c85eee 100644 --- a/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java +++ b/samza-api/src/main/java/org/apache/samza/storage/StorageEngine.java @@ -43,11 +43,15 @@ public interface StorageEngine { * provided in one {@link java.util.Iterator} and not deserialized for * efficiency, allowing the implementation to optimize replay, if possible. * + * The implementers are expected to handle interrupt signals to the restoration thread and rethrow the exception to + * upstream so that {@code TaskRestoreManager} can accordingly notify the container. + * * @param envelopes * An iterator of envelopes that the storage engine can read from to * restore its state on startup. + * @throws InterruptedException when received interrupts during restoration */ - void restore(ChangelogSSPIterator envelopes); + void restore(ChangelogSSPIterator envelopes) throws InterruptedException; /** * Flush any cached messages diff --git a/samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java b/samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java index f952f6eff5..5a395207d0 100644 --- a/samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java +++ b/samza-core/src/main/java/org/apache/samza/processor/StreamProcessor.java @@ -85,16 +85,18 @@ * * Describes the valid state transitions of the {@link StreamProcessor}. * - * - * ──────────────────────────────── - * │ │ - * │ │ - * │ │ - * │ │ - * New StreamProcessor.start() Rebalance triggered V Receives JobModel │ - * StreamProcessor ──────────▶ NEW ───────────────────────────▶ STARTED ──────────────────▶ IN_REBALANCE ─────────────────────▶ RUNNING - * Creation │ │ by group leader │ and starts Container │ - * │ │ │ │ + * Receives another re-balance request when the container + * from the previous re-balance is still in INIT phase + * ──────────────────────────────────────────────── + * │ │ │ + * │ │ │ + * │ │ │ + * │ │ │ + * New StreamProcessor.start() Rebalance triggered V Receives JobModel │ │ + * StreamProcessor ──────────▶ NEW ───────────────────────────▶ STARTED ──────────────────▶ IN_REBALANCE ─────────────────────▶ RUNNING │ + * Creation │ │ by group leader │ and starts │Container │ │ + * │ │ │ │ │ │ + * │ │ │ ─────────────────────────────── * Stre│amProcessor.stop() Stre│amProcessor.stop() Stre│amProcessor.stop() Stre│amProcessor.stop() * │ │ │ │ * │ │ │ │ @@ -133,7 +135,6 @@ public class StreamProcessor { private final Config config; private final long taskShutdownMs; private final String processorId; - private final ExecutorService containerExcecutorService; private final Object lock = new Object(); private final MetricsRegistryMap metricsRegistry; private final MetadataStore metadataStore; @@ -176,6 +177,9 @@ public State getState() { @VisibleForTesting JobCoordinatorListener jobCoordinatorListener = null; + @VisibleForTesting + ExecutorService containerExecutorService; + /** * Same as {@link #StreamProcessor(String, Config, Map, TaskFactory, ProcessorLifecycleListener, JobCoordinator)}, except * it creates a {@link JobCoordinator} instead of accepting it as an argument. @@ -288,11 +292,15 @@ public StreamProcessor(String processorId, Config config, Map @@ -348,7 +356,7 @@ public void stop() { boolean hasContainerShutdown = stopSamzaContainer(); if (!hasContainerShutdown) { LOGGER.info("Interrupting the container: {} thread to die.", container); - containerExcecutorService.shutdownNow(); + containerExecutorService.shutdownNow(); } } catch (Throwable throwable) { LOGGER.error(String.format("Exception occurred on container: %s shutdown of stream processor: %s.", container, processorId), throwable); @@ -444,6 +452,19 @@ private boolean stopSamzaContainer() { return hasContainerShutdown; } + private boolean interruptContainerAndShutdownExecutorService() { + try { + containerExecutorService.shutdownNow(); + containerShutdownLatch.await(taskShutdownMs, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + LOGGER.info("Received an interrupt during interrupting container. Proceeding to check if the container callback " + + "decremented the shutdown latch. "); + } + + // we call interrupt successful as long as the shut down latch is decremented by the container call back. + return containerShutdownLatch.getCount() == 0; + } + private JobCoordinatorListener createJobCoordinatorListener() { return new JobCoordinatorListener() { @@ -461,8 +482,23 @@ public void onJobModelExpired() { } else { LOGGER.info("Container: {} shutdown completed for stream processor: {}.", container, processorId); } + } else if (state == State.IN_REBALANCE) { + if (container != null) { + boolean hasContainerShutdown = interruptContainerAndShutdownExecutorService(); + if (!hasContainerShutdown) { + LOGGER.warn("Job model expire unsuccessful. Failed to interrupt container: {} safely. " + + "Stopping the stream processor: {}", container, processorId); + state = State.STOPPING; + jobCoordinator.stop(); + } else { + containerExecutorService = createExecutorService(); + } + } else { + LOGGER.info("Ignoring Job model expired since a rebalance is already in progress"); + } } else { - LOGGER.info("Ignoring onJobModelExpired invocation since the current state is {} and not in {}.", state, ImmutableList.of(State.RUNNING, State.STARTED)); + LOGGER.info("Ignoring onJobModelExpired invocation since the current state is {} and not in {}.", state, + ImmutableList.of(State.RUNNING, State.STARTED, State.IN_REBALANCE)); } } } @@ -475,7 +511,7 @@ public void onNewJobModel(String processorId, JobModel jobModel) { container = createSamzaContainer(processorId, jobModel); container.setContainerListener(new ContainerListener()); LOGGER.info("Starting the container: {} for the stream processor: {}.", container, processorId); - containerExcecutorService.submit(container); + containerExecutorService.submit(container); } else { LOGGER.info("Ignoring onNewJobModel invocation since the current state is {} and not {}.", state, State.IN_REBALANCE); } @@ -490,7 +526,7 @@ public void onCoordinatorStop() { // we only want to interrupt when container shutdown times out. if (!hasContainerShutdown) { - containerExcecutorService.shutdownNow(); + containerExecutorService.shutdownNow(); } state = State.STOPPED; } @@ -508,7 +544,7 @@ public void onCoordinatorFailure(Throwable throwable) { // we only want to interrupt when container shutdown times out. if (!hasContainerShutdown) { - containerExcecutorService.shutdownNow(); + containerExecutorService.shutdownNow(); } state = State.STOPPED; } diff --git a/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java index 70c61746e8..d2f009798a 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java @@ -316,7 +316,7 @@ private String getStartingOffset(SystemStreamPartition systemStreamPartition, Sy * Restore each store in taskStoresToRestore sequentially */ @Override - public void restore() { + public void restore() throws InterruptedException { for (String storeName : taskStoresToRestore) { LOG.info("Restoring store: {} for task: {}", storeName, taskModel.getTaskName()); SystemConsumer systemConsumer = storeConsumers.get(storeName); diff --git a/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java b/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java index 2cbeddcdf4..11237a8ae6 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java +++ b/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java @@ -117,7 +117,14 @@ public void run() { systemAdmins.start(); this.containerStorageManagers.forEach((containerName, containerStorageManager) -> { - containerStorageManager.start(); + try { + containerStorageManager.start(); + } catch (InterruptedException e) { + // we can ignore the exception since its only used in the context of a command line tool and bubbling the + // exception upstream isn't needed. + LOG.warn("Received an interrupt during store restoration for container {}." + + " Proceeding with the next container", containerName); + } }); this.containerStorageManagers.forEach((containerName, containerStorageManager) -> { containerStorageManager.shutdown(); diff --git a/samza-core/src/main/java/org/apache/samza/storage/TaskRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/TaskRestoreManager.java index 2bdeeead05..f60e14876b 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/TaskRestoreManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/TaskRestoreManager.java @@ -35,8 +35,17 @@ public interface TaskRestoreManager { /** * Restore state from checkpoints, state snapshots and changelog. + * Currently, store restoration happens on a separate thread pool within {@code ContainerStorageManager}. In case of + * interrupt/shutdown signals from {@code SamzaContainer}, {@code ContainerStorageManager} may interrupt the restore + * thread. + * + * Note: Typically, interrupt signals don't bubble up as {@link InterruptedException} unless the restore thread is + * waiting on IO/network. In case of busy looping, implementors are expected to check the interrupt status of the + * thread periodically and shutdown gracefully before throwing {@link InterruptedException} upstream. + * {@code SamzaContainer} will not wait for clean up and the interrupt signal is the best effort by the container + * to notify that its shutting down. */ - void restore(); + void restore() throws InterruptedException; /** * Stop all persistent stores after restoring. diff --git a/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java index 689d431cbd..8133fac7be 100644 --- a/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java +++ b/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java @@ -117,7 +117,7 @@ public void init(Map checkpointedChangelogOffsets } @Override - public void restore() { + public void restore() throws InterruptedException { Map storesToRestore = storeActions.storesToRestore; for (Map.Entry entry : storesToRestore.entrySet()) { diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala index d15c1090f2..092175da32 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala @@ -27,7 +27,6 @@ import java.time.Duration import java.util import java.util.{Base64, Optional} import java.util.concurrent.{ExecutorService, Executors, ScheduledExecutorService, TimeUnit} -import java.util.stream.Collectors import com.google.common.annotations.VisibleForTesting import com.google.common.util.concurrent.ThreadFactoryBuilder @@ -769,6 +768,27 @@ class SamzaContainer( else Thread.sleep(Long.MaxValue) } catch { + case e: InterruptedException => + /* + * We don't want to categorize interrupts as failure since the only place the container thread gets interrupted within + * our code inside stream processor is during the following two scenarios + * 1. During a re-balance, if the container has not started or hasn't reported start status to StreamProcessor. + * Subsequently stream processor attempts to interrupt the container thread before proceeding to join the barrier + * to agree on the new work assignment. + * 2. During shutdown signals to stream processor (external or internal), the stream processor signals the container to + * shutdown and waits for `task.shutdown.ms` before forcefully shutting down the container executor service which in + * turn interrupts the container thread. + * + * In the both of these scenarios, the failure cause is either captured externally (timing out scenario) or internally + * (failed attempt to shut down the container). The act of interrupting the container thread is an explicit intent to shutdown + * the container since it is not capable of reacting to shutdown signals in all scenarios. + * + */ + if (status.equals(SamzaContainerStatus.STARTED)) { + warn("Received an interrupt in run loop.", e) + } else { + warn("Received an interrupt during initialization.", e) + } case e: Throwable => if (status.equals(SamzaContainerStatus.STARTED)) { error("Caught exception/error in run loop.", e) diff --git a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java index afd3e69bd8..8623e5d9c9 100644 --- a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java +++ b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java @@ -632,7 +632,7 @@ private Set getSideInputStorageManagers() { return this.sideInputStorageManagers.values().stream().collect(Collectors.toSet()); } - public void start() throws SamzaException { + public void start() throws SamzaException, InterruptedException { Map checkpointedChangelogSSPOffsets = new HashMap<>(); if (new TaskConfig(config).getTransactionalStateRestoreEnabled()) { getTasks(containerModel, TaskMode.Active).forEach((taskName, taskModel) -> { @@ -657,7 +657,8 @@ public void start() throws SamzaException { } // Restoration of all stores, in parallel across tasks - private void restoreStores(Map checkpointedChangelogSSPOffsets) { + private void restoreStores(Map checkpointedChangelogSSPOffsets) + throws InterruptedException { LOG.info("Store Restore started"); // initialize each TaskStorageManager @@ -665,7 +666,7 @@ private void restoreStores(Map checkpointedChange taskStorageManager.init(checkpointedChangelogSSPOffsets)); // Start each store consumer once - this.storeConsumers.values().stream().distinct().forEach(systemConsumer -> systemConsumer.start()); + this.storeConsumers.values().stream().distinct().forEach(SystemConsumer::start); // Create a thread pool for parallel restores (and stopping of persistent stores) ExecutorService executorService = Executors.newFixedThreadPool(this.parallelRestoreThreadPoolSize, @@ -684,6 +685,11 @@ private void restoreStores(Map checkpointedChange for (Future future : taskRestoreFutures) { try { future.get(); + } catch (InterruptedException e) { + LOG.warn("Received an interrupt during store restoration. Issuing interrupts to the store restoration workers to exit " + + "prematurely without restoring full state."); + executorService.shutdownNow(); + throw e; } catch (Exception e) { LOG.error("Exception when restoring ", e); throw new SamzaException("Exception when restoring ", e); @@ -693,7 +699,7 @@ private void restoreStores(Map checkpointedChange executorService.shutdown(); // Stop each store consumer once - this.storeConsumers.values().stream().distinct().forEach(systemConsumer -> systemConsumer.stop()); + this.storeConsumers.values().stream().distinct().forEach(SystemConsumer::stop); // Now re-create persistent stores in read-write mode, leave non-persistent stores as-is recreatePersistentTaskStoresInReadWriteMode(this.containerModel, jobContext, containerContext, @@ -791,6 +797,14 @@ public void run() { } } catch (InterruptedException e) { + LOG.warn("Received an interrupt during side inputs store restoration." + + " Exiting prematurely without completing store restore."); + /* + * We want to stop side input restoration and rethrow the exception upstream. Container should handle the + * interrupt exception and shutdown the components and cleaning up the resource. We don't want to clean up the + * resources prematurely here. + */ + shouldShutdown = true; // todo: should we cancel the flush future right away or wait for container to handle it as part of shutdown sequence? throw new SamzaException("Side inputs read was interrupted", e); } @@ -920,22 +934,32 @@ public TaskRestoreCallable(SamzaContainerMetrics samzaContainerMetrics, TaskName @Override public Void call() { long startTime = System.currentTimeMillis(); - LOG.info("Starting stores in task instance {}", this.taskName.getTaskName()); - taskRestoreManager.restore(); - - // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted - // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation). - taskRestoreManager.stopPersistentStores(); - - long timeToRestore = System.currentTimeMillis() - startTime; - - if (this.samzaContainerMetrics != null) { - Gauge taskGauge = this.samzaContainerMetrics.taskStoreRestorationMetrics().getOrDefault(this.taskName, null); - - if (taskGauge != null) { - taskGauge.set(timeToRestore); + try { + LOG.info("Starting stores in task instance {}", this.taskName.getTaskName()); + taskRestoreManager.restore(); + } catch (InterruptedException e) { + /* + * The container thread is the only external source to trigger an interrupt to the restoration thread and thus + * it is okay to swallow this exception and not propagate it upstream. If the container is interrupted during + * the store restoration, ContainerStorageManager signals the restore workers to abandon restoration and then + * finally propagates the exception upstream to trigger container shutdown. + */ + LOG.warn("Received an interrupt during store restoration for task: {}.", this.taskName.getTaskName()); + } finally { + // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted + // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation). + taskRestoreManager.stopPersistentStores(); + long timeToRestore = System.currentTimeMillis() - startTime; + + if (this.samzaContainerMetrics != null) { + Gauge taskGauge = this.samzaContainerMetrics.taskStoreRestorationMetrics().getOrDefault(this.taskName, null); + + if (taskGauge != null) { + taskGauge.set(timeToRestore); + } } } + return null; } } diff --git a/samza-core/src/main/scala/org/apache/samza/system/SystemConsumers.scala b/samza-core/src/main/scala/org/apache/samza/system/SystemConsumers.scala index b41c2454d8..580fd605d8 100644 --- a/samza-core/src/main/scala/org/apache/samza/system/SystemConsumers.scala +++ b/samza-core/src/main/scala/org/apache/samza/system/SystemConsumers.scala @@ -137,6 +137,13 @@ class SystemConsumers ( */ private val emptySystemStreamPartitionsBySystem = new HashMap[String, Set[SystemStreamPartition]]() + /** + * Denotes if the SystemConsumers have started. The flag is useful in the event of shutting down since interrupt + * on Samza Container will shutdown components and container currently doesn't track what components have started + * successfully. + */ + private var started = false + /** * Default timeout to noNewMessagesTimeout. Every time SystemConsumers * receives incoming messages, it sets timeout to 0. Every time @@ -185,15 +192,23 @@ class SystemConsumers ( chooser.start + started = true + refresh } def stop { - debug("Stopping consumers.") + if (started) { + debug("Stopping consumers.") - consumers.values.foreach(_.stop) + consumers.values.foreach(_.stop) - chooser.stop + chooser.stop + + started = false + } else { + debug("Ignoring the consumers stop request since it never started.") + } } diff --git a/samza-core/src/main/scala/org/apache/samza/system/SystemProducers.scala b/samza-core/src/main/scala/org/apache/samza/system/SystemProducers.scala index e8ce961af0..1744195fab 100644 --- a/samza-core/src/main/scala/org/apache/samza/system/SystemProducers.scala +++ b/samza-core/src/main/scala/org/apache/samza/system/SystemProducers.scala @@ -35,16 +35,29 @@ class SystemProducers( */ dropSerializationError: Boolean = false) extends Logging { + /** + * Denotes if the SystemConsumers have started. The flag is useful in the event of shutting down since interrupt + * on Samza Container will shutdown components and container currently doesn't track what components have started + * successfully. + */ + private var started = false + def start { debug("Starting producers.") producers.values.foreach(_.start) + started = true } def stop { - debug("Stopping producers.") + if (started) { + debug("Stopping producers.") - producers.values.foreach(_.stop) + producers.values.foreach(_.stop) + started = false + } else { + debug("Ignoring the producers stop request since it never started.") + } } def register(source: String) { diff --git a/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java b/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java index 6d78b77d10..0978738eda 100644 --- a/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java +++ b/samza-core/src/test/java/org/apache/samza/processor/TestStreamProcessor.java @@ -27,6 +27,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; @@ -55,6 +56,7 @@ import org.powermock.api.mockito.PowerMockito; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; @@ -467,13 +469,67 @@ public void testOnJobModelExpiredShouldMakeCorrectStateTransitions() { assertEquals(State.STOPPING, streamProcessor.getState()); Mockito.verify(mockSamzaContainer, Mockito.times(1)).shutdown(); Mockito.verify(mockJobCoordinator, Mockito.times(1)).stop(); + } - // If StreamProcessor is in IN_REBALANCE state, onJobModelExpired should be a NO_OP. - streamProcessor.state = State.IN_REBALANCE; + @Test + public void testJobModelExpiredDuringAnExistingRebalance() { + JobCoordinator mockJobCoordinator = Mockito.mock(JobCoordinator.class); + ProcessorLifecycleListener lifecycleListener = Mockito.mock(ProcessorLifecycleListener.class); + ExecutorService mockExecutorService = Mockito.mock(ExecutorService.class); + MapConfig config = new MapConfig(ImmutableMap.of("task.shutdown.ms", "0")); + StreamProcessor streamProcessor = new StreamProcessor("TestProcessorId", config, new HashMap<>(), null, + Optional.empty(), Optional.empty(), Optional.empty(), sp -> lifecycleListener, mockJobCoordinator, Mockito.mock(MetadataStore.class)); - streamProcessor.jobCoordinatorListener.onJobModelExpired(); + runJobModelExpireDuringRebalance(streamProcessor, mockExecutorService, false); assertEquals(State.IN_REBALANCE, streamProcessor.state); + assertNotEquals(mockExecutorService, streamProcessor.containerExecutorService); + Mockito.verify(mockExecutorService, Mockito.times(1)).shutdownNow(); + } + + @Test + public void testJobModelExpiredDuringAnExistingRebalanceWithContainerInterruptFailed() { + JobCoordinator mockJobCoordinator = Mockito.mock(JobCoordinator.class); + ProcessorLifecycleListener lifecycleListener = Mockito.mock(ProcessorLifecycleListener.class); + ExecutorService mockExecutorService = Mockito.mock(ExecutorService.class); + MapConfig config = new MapConfig(ImmutableMap.of("task.shutdown.ms", "0")); + StreamProcessor streamProcessor = new StreamProcessor("TestProcessorId", config, new HashMap<>(), null, + Optional.empty(), Optional.empty(), Optional.empty(), sp -> lifecycleListener, mockJobCoordinator, Mockito.mock(MetadataStore.class)); + + runJobModelExpireDuringRebalance(streamProcessor, mockExecutorService, true); + + assertEquals(State.STOPPING, streamProcessor.state); + assertEquals(mockExecutorService, streamProcessor.containerExecutorService); + Mockito.verify(mockExecutorService, Mockito.times(1)).shutdownNow(); + Mockito.verify(mockJobCoordinator, Mockito.times(1)).stop(); + } + + private void runJobModelExpireDuringRebalance(StreamProcessor streamProcessor, ExecutorService executorService, + boolean failContainerInterrupt) { + SamzaContainer mockSamzaContainer = Mockito.mock(SamzaContainer.class); + CountDownLatch shutdownLatch = new CountDownLatch(1); + + /* + * When there is an initialized container that hasn't started and the stream processor is still in re-balance phase, + * subsequent job model expire request should attempt to interrupt the existing container to safely shut it down + * before proceeding to join the barrier. As part of safe shutdown sequence, we want to ensure shutdownNow is invoked + * on the existing executorService to signal interrupt and make sure new executor service is created. + */ + + Mockito.when(executorService.shutdownNow()).thenAnswer(ctx -> { + if (!failContainerInterrupt) { + shutdownLatch.countDown(); + } + return null; + }); + Mockito.when(executorService.isShutdown()).thenReturn(true); + + streamProcessor.state = State.IN_REBALANCE; + streamProcessor.container = mockSamzaContainer; + streamProcessor.containerExecutorService = executorService; + streamProcessor.containerShutdownLatch = shutdownLatch; + + streamProcessor.jobCoordinatorListener.onJobModelExpired(); } @Test diff --git a/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala b/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala index 78136bf03c..69223df290 100644 --- a/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala +++ b/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala @@ -35,6 +35,8 @@ import org.junit.Assert._ import org.junit.{Before, Test} import org.mockito.Matchers.{any, notNull} import org.mockito.Mockito._ +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer import org.mockito.{Mock, Mockito, MockitoAnnotations} import org.scalatest.junit.AssertionsForJUnit import org.scalatest.mockito.MockitoSugar @@ -139,6 +141,46 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar { verify(this.runLoop).run() } + @Test + def testInterruptDuringStoreRestorationShutdownContainer(): Unit = { + when(this.containerStorageManager.start()) + .thenAnswer(new Answer[Void] { + override def answer(mock: InvocationOnMock): Void = { + Thread.sleep(1000) + throw new InterruptedException("Injecting interrupt into container storage manager") + } + }) + + this.samzaContainer.run + + assertEquals(SamzaContainerStatus.STOPPED, this.samzaContainer.getStatus()) + verify(this.samzaContainerListener).beforeStart() + verify(this.samzaContainerListener).afterStop() + verify(this.samzaContainerListener, never()).afterFailure(any()) + verify(this.runLoop, times(0)).run() + } + + @Test + def testInterruptDuringStoreRestorationWithErrorsDuringContainerShutdown(): Unit = { + when(this.containerStorageManager.start()) + .thenAnswer(new Answer[Void] { + override def answer(mock: InvocationOnMock): Void = { + Thread.sleep(1000) + throw new InterruptedException("Injecting interrupt into container storage manager") + } + }) + + when(this.taskInstance.shutdownTask).thenThrow(new RuntimeException("Trigger a shutdown, please.")) + + this.samzaContainer.run + + assertEquals(SamzaContainerStatus.FAILED, this.samzaContainer.getStatus()) + verify(this.samzaContainerListener).beforeStart() + verify(this.samzaContainerListener).afterFailure(any()) + verify(this.samzaContainerListener, never()).afterStop() + verify(this.runLoop, times(0)).run() + } + @Test def testFailureDuringShutdown(): Unit = { doNothing().when(this.runLoop).run() // run loop completes successfully diff --git a/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java b/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java index 43872e906e..bdc7e0ee72 100644 --- a/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java +++ b/samza-core/src/test/scala/org/apache/samza/storage/TestContainerStorageManager.java @@ -101,7 +101,7 @@ private void addMockedTask(String taskname, int changelogPartition) { * Method to create a containerStorageManager with mocked dependencies */ @Before - public void setUp() { + public void setUp() throws InterruptedException { taskRestoreMetricGauges = new HashMap<>(); this.tasks = new HashMap<>(); this.taskInstanceMetrics = new HashMap<>(); @@ -243,7 +243,7 @@ public Void answer(InvocationOnMock invocation) { } @Test - public void testParallelismAndMetrics() { + public void testParallelismAndMetrics() throws InterruptedException { this.containerStorageManager.start(); this.containerStorageManager.shutdown(); diff --git a/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngine.scala b/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngine.scala index bc6778eed4..afba824d3a 100644 --- a/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngine.scala +++ b/samza-kv/src/main/scala/org/apache/samza/storage/kv/KeyValueStorageEngine.scala @@ -124,7 +124,7 @@ class KeyValueStorageEngine[K, V]( val batch = new java.util.ArrayList[Entry[Array[Byte], Array[Byte]]](batchSize) var lastBatchFlushed = false - while(iterator.hasNext) { + while(iterator.hasNext && !Thread.currentThread().isInterrupted) { val envelope = iterator.next() val keyBytes = envelope.getKey.asInstanceOf[Array[Byte]] val valBytes = envelope.getMessage.asInstanceOf[Array[Byte]] @@ -200,6 +200,11 @@ class KeyValueStorageEngine[K, V]( // flush the store and the changelog producer flush() // TODO HIGH pmaheshw SAMZA-2338: Need a way to flush changelog producers. This only flushes the stores. + + if (Thread.currentThread().isInterrupted) { + warn("Received an interrupt during store restoration. Exiting without restoring the full state.") + throw new InterruptedException("Received an interrupt during store restoration.") + } } def flush() = { diff --git a/samza-kv/src/test/scala/org/apache/samza/storage/kv/TestKeyValueStorageEngine.scala b/samza-kv/src/test/scala/org/apache/samza/storage/kv/TestKeyValueStorageEngine.scala index 25445c2912..151fd846a4 100644 --- a/samza-kv/src/test/scala/org/apache/samza/storage/kv/TestKeyValueStorageEngine.scala +++ b/samza-kv/src/test/scala/org/apache/samza/storage/kv/TestKeyValueStorageEngine.scala @@ -21,8 +21,10 @@ package org.apache.samza.storage.kv import java.io.File import java.util.Arrays +import java.util.concurrent.{Callable, ExecutionException, ExecutorService, Executors} import org.apache.samza.Partition +import org.apache.samza.context.Context import org.apache.samza.storage.StoreProperties import org.apache.samza.system.ChangelogSSPIterator.Mode import org.apache.samza.system.{ChangelogSSPIterator, IncomingMessageEnvelope, SystemStreamPartition} @@ -30,6 +32,8 @@ import org.apache.samza.task.MessageCollector import org.junit.Assert._ import org.junit.{After, Before, Test} import org.mockito.Mockito._ +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer class TestKeyValueStorageEngine { var engine: KeyValueStorageEngine[String, String] = null @@ -163,6 +167,50 @@ class TestKeyValueStorageEngine { assertEquals(15, metrics.restoredBytesGauge.getValue) // 3 keys * 2 bytes/key + 3 msgs * 3 bytes/msg } + @Test + def testRestoreInterruptedThrowsInterruptException(): Unit = { + val changelogSSP = new SystemStreamPartition("TestSystem", "TestStream", new Partition(0)) + val iterator = mock(classOf[ChangelogSSPIterator]) + val executorService = Executors.newSingleThreadExecutor() + val restore = new Callable[Void] { + override def call(): Void = { + engine.restore(iterator) + null + } + } + + when(iterator.hasNext) + .thenReturn(true) + .thenReturn(true) + .thenAnswer(new Answer[Boolean] { + override def answer(invocation: InvocationOnMock): Boolean = { + executorService.shutdownNow() + true + } + }) + .thenReturn(false) + when(iterator.next()) + .thenReturn(new IncomingMessageEnvelope(changelogSSP, "0", Array[Byte](1, 2), Array[Byte](3, 4, 5))) + .thenReturn(new IncomingMessageEnvelope(changelogSSP, "1", Array[Byte](2, 3), Array[Byte](4, 5, 6))) + .thenReturn(new IncomingMessageEnvelope(changelogSSP, "2", Array[Byte](3, 4), Array[Byte](5, 6, 7))) + when(iterator.getMode) + .thenReturn(Mode.RESTORE) + .thenReturn(Mode.RESTORE) + .thenReturn(Mode.RESTORE) + + try { + val restoreFuture = executorService.submit(restore) + restoreFuture.get() + fail("Expected execution exception during restoration") + } catch { + case e: ExecutionException => { + assertTrue(e.getCause.isInstanceOf[InterruptedException]) + // Make sure we don't restore any more records and bail out + assertEquals(2, metrics.restoredMessagesGauge.getValue) + } + } + } + @Test(expected = classOf[IllegalStateException]) def testThrowsIfIteratorModeChangesFromTrimToRestore(): Unit = { val changelogSSP = new SystemStreamPartition("TestSystem", "TestStream", new Partition(0)) From 1854e3aec73e44a1c6bc14fbe85110bf34f28548 Mon Sep 17 00:00:00 2001 From: rmatharu <40646191+rmatharu@users.noreply.github.com> Date: Tue, 10 Dec 2019 14:36:53 -0800 Subject: [PATCH 18/49] Adding null-check before incrementing metrics for bytesSerialized (#1234) --- .../org/apache/samza/storage/kv/SerializedKeyValueStore.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/samza-kv/src/main/scala/org/apache/samza/storage/kv/SerializedKeyValueStore.scala b/samza-kv/src/main/scala/org/apache/samza/storage/kv/SerializedKeyValueStore.scala index 51df502d3a..d220fc79a6 100644 --- a/samza-kv/src/main/scala/org/apache/samza/storage/kv/SerializedKeyValueStore.scala +++ b/samza-kv/src/main/scala/org/apache/samza/storage/kv/SerializedKeyValueStore.scala @@ -136,7 +136,9 @@ class SerializedKeyValueStore[K, V]( null } else { val bytes = serde.toBytes(t) - metrics.bytesSerialized.inc(bytes.size) + if (bytes != null) { + metrics.bytesSerialized.inc(bytes.size) + } bytes } From b53cb146bf63489f2b80c4adb37759423ec2b38a Mon Sep 17 00:00:00 2001 From: Cameron Lee Date: Mon, 16 Dec 2019 11:10:02 -0800 Subject: [PATCH 19/49] SAMZA-2418: Integration tests are failing and do not properly pass params to zopkio (#1235) --- bin/integration-tests.sh | 3 +- build.gradle | 4 +-- docs/contribute/tests.md | 6 ++-- samza-test/src/main/python/configs/tests.json | 2 +- .../src/main/python/stream_processor.py | 2 +- samza-test/src/main/resources/log4j.xml | 27 ----------------- samza-test/src/main/resources/log4j2.xml | 29 +++++++++++++++++++ 7 files changed, 36 insertions(+), 37 deletions(-) delete mode 100644 samza-test/src/main/resources/log4j.xml create mode 100644 samza-test/src/main/resources/log4j2.xml diff --git a/bin/integration-tests.sh b/bin/integration-tests.sh index 248236fd34..b0e99278ec 100755 --- a/bin/integration-tests.sh +++ b/bin/integration-tests.sh @@ -72,8 +72,7 @@ source $SAMZA_INTEGRATION_TESTS_DIR/bin/activate pip install -r $SCRIPTS_DIR/requirements.txt # treat all trailing parameters (after dirname, test_type) as zopkio switches -shift -SWITCHES="${*:3}" +SWITCHES="${@:3}" # default to info-level debugging if not specified if [[ $SWITCHES != *"console-log-level"* ]]; then diff --git a/build.gradle b/build.gradle index dbf777ea7a..c6452ac2c4 100644 --- a/build.gradle +++ b/build.gradle @@ -845,9 +845,6 @@ project(":samza-test_$scalaSuffix") { compile.exclude group: 'javax.jms', module: 'jms' compile.exclude group: 'com.sun.jdmk', module: 'jmxtools' compile.exclude group: 'com.sun.jmx', module: 'jmxri' - // Exclude Slf4j bindings for log4j1 and log4j2 since samza-test just logs to console using slf4j-simple - compile.exclude group: 'org.slf4j', module: 'slf4j-log4j12' - compile.exclude group: 'org.apache.logging.log4j', module: 'log4j-slf4j-impl' } dependencies { @@ -858,6 +855,7 @@ project(":samza-test_$scalaSuffix") { compile project(":samza-core_$scalaSuffix") compile project(":samza-kafka_$scalaSuffix") compile project(":samza-sql_$scalaSuffix") + runtime project(":samza-log4j2_$scalaSuffix") runtime project(":samza-yarn_$scalaSuffix") runtime project(":samza-hdfs_$scalaSuffix") compile "org.scala-lang:scala-library:$scalaVersion" diff --git a/docs/contribute/tests.md b/docs/contribute/tests.md index 9b947653d3..964e090c68 100644 --- a/docs/contribute/tests.md +++ b/docs/contribute/tests.md @@ -60,7 +60,7 @@ On Mac, check-all.sh will default to the appropriate path for each environment v Samza uses [Zopkio](https://github.com/linkedin/Zopkio) to deploy and execute its integration tests. Integration tests can be executed by running: - ./bin/integration-tests.sh /tmp/samza-tests + ./bin/integration-tests.sh /tmp/samza-tests [yarn-integration-tests|standalone-integration-tests] The parameter defines where the integration tests should install packages both locally and on remote systems. Executing this command will: @@ -90,7 +90,7 @@ Zopkio supports public key authentication if you prefer to use it, or if your en Once this is done, you can run Zopkio with the \-\-nopassword parameter: - ./bin/integration-tests.sh /tmp/samza-tests --nopassword + ./bin/integration-tests.sh /tmp/samza-tests [yarn-integration-tests|standalone-integration-tests] --nopassword This will skip the password prompt, and force Zopkio to try public key authentication. @@ -98,7 +98,7 @@ This will skip the password prompt, and force Zopkio to try public key authentic The integration-tests.sh script will set the console log level to INFO by default. The level can be changed with: - ./bin/integration-tests.sh /tmp/samza-tests --console-log-level DEBUG + ./bin/integration-tests.sh /tmp/samza-tests [yarn-integration-tests|standalone-integration-tests] --console-log-level DEBUG Changing this setting will define how verbose Zopkio is during test execution. It does not affect any of the log4j.xml settings in Samza, YARN, Kafka, or ZooKeeper. diff --git a/samza-test/src/main/python/configs/tests.json b/samza-test/src/main/python/configs/tests.json index 0b0ab92338..e89c614977 100644 --- a/samza-test/src/main/python/configs/tests.json +++ b/samza-test/src/main/python/configs/tests.json @@ -1,5 +1,5 @@ { - "samza_executable": "samza-test_2.11-1.1.1-SNAPSHOT.tgz", + "samza_executable": "samza-test_2.11-1.4.0-SNAPSHOT.tgz", "samza_install_path": "deploy/smoke_tests", "samza_config_factory": "org.apache.samza.config.factories.PropertiesConfigFactory" } diff --git a/samza-test/src/main/python/stream_processor.py b/samza-test/src/main/python/stream_processor.py index 6d4353ea17..4511ef8139 100644 --- a/samza-test/src/main/python/stream_processor.py +++ b/samza-test/src/main/python/stream_processor.py @@ -43,7 +43,7 @@ def __init__(self, host_name, processor_id): logger.info('Running processor start command: {0}'.format(self.processor_start_command)) self.deployment_config = { 'install_path': os.path.join(runtime.get_active_config('remote_install_path'), 'deploy/{0}'.format(self.processor_id)), - 'executable': 'samza-test_2.11-1.1.1-SNAPSHOT.tgz', + 'executable': 'samza-test_2.11-1.4.0-SNAPSHOT.tgz', 'post_install_cmds': [], 'start_command': self.processor_start_command, 'stop_command': '', diff --git a/samza-test/src/main/resources/log4j.xml b/samza-test/src/main/resources/log4j.xml deleted file mode 100644 index 12f752ca8d..0000000000 --- a/samza-test/src/main/resources/log4j.xml +++ /dev/null @@ -1,27 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/samza-test/src/main/resources/log4j2.xml b/samza-test/src/main/resources/log4j2.xml new file mode 100644 index 0000000000..0d60ed4e70 --- /dev/null +++ b/samza-test/src/main/resources/log4j2.xml @@ -0,0 +1,29 @@ + + + + + + + + + + + + + + + + + + + + From 404a0fc61b593e0296f2a1c44bde8642ab100e59 Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Tue, 17 Dec 2019 17:24:23 -0800 Subject: [PATCH 20/49] SAMZA-2407: Add job.config.loader.* configs in JobConfig & corresponding env variables in ShellCommandConfig Design: https://cwiki.apache.org/confluence/display/SAMZA/SEP-23%3A+Simplify+Job+Runner Changes: 1. Add job.config.loader.factory to specify ConfigLoaderFactory. 2. Add job.config.loader.properties prefix to specify properties needed by ConfigLoader to fetch config. 3. Add ENV_SUBMISSION_CONFIG enviroment variable to pass job submission configs to Yarn. API Changes: N/A. This is part of a series PRs, detailed information will be provided in the last/main PR. Upgrade Instructions: N/A. This is part of a series PRs, detailed information will be provided in the last/main PR. Usage Instructions: N/A. This is part of a series PRs, detailed information will be provided in the last/main PR. Tests: N/A. --- .../org/apache/samza/config/JobConfig.java | 76 +++++++++++++++---- .../samza/config/ShellCommandConfig.scala | 9 ++- 2 files changed, 68 insertions(+), 17 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/config/JobConfig.java b/samza-core/src/main/java/org/apache/samza/config/JobConfig.java index 1d94f95f38..a64cc9f839 100644 --- a/samza-core/src/main/java/org/apache/samza/config/JobConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/JobConfig.java @@ -47,6 +47,15 @@ public class JobConfig extends MapConfig { public static final String CONFIG_REWRITERS = "job.config.rewriters"; public static final String CONFIG_REWRITER_CLASS = "job.config.rewriter.%s.class"; + /** + * job.config.loader.factory specifies {@link ConfigLoaderFactory} to get {@link ConfigLoader} + */ + public static final String CONFIG_LOADER_FACTORY = "job.config.loader.factory"; + /** + * job.config.loader.properties prefix wraps properties needed for {@link ConfigLoader} to loader config. + */ + public static final String CONFIG_LOADER_PROPERTIES_PREFIX = "job.config.loader.properties."; + public static final String JOB_NAME = "job.name"; public static final String JOB_ID = "job.id"; static final String DEFAULT_JOB_ID = "1"; @@ -125,6 +134,16 @@ public class JobConfig extends MapConfig { public static final String CONTAINER_METADATA_FILENAME_FORMAT = "%s.metadata"; // Filename: .metadata public static final String CONTAINER_METADATA_DIRECTORY_SYS_PROPERTY = "samza.log.dir"; + + // Auto-sizing related configs tthat ake precedence over respective sizing confings job.container.count, etc, + // *only* when job.autosizing.enabled is true. Otherwise current behavior is maintained. + public static final String JOB_AUTOSIZING_ENABLED = "job.autosizing.enabled"; + public static final String JOB_AUTOSIZING_CONTAINER_COUNT = "job.autosizing.container.count"; + public static final String JOB_AUTOSIZING_CONTAINER_THREAD_POOL_SIZE = "job.autosizing.container.thread.pool.size"; + public static final String JOB_AUTOSIZING_CONTAINER_MAX_HEAP_MB = "job.autosizing.container.maxheap.mb"; + public static final String JOB_AUTOSIZING_CONTAINER_MEMORY_MB = "job.autosizing.container.memory.mb"; + public static final String JOB_AUTOSIZING_CONTAINER_MAX_CORES = "job.autosizing.container.cpu.cores"; + public static final String COORDINATOR_STREAM_FACTORY = "job.coordinatorstream.config.factory"; public static final String DEFAULT_COORDINATOR_STREAM_CONFIG_FACTORY = "org.apache.samza.util.DefaultCoordinatorStreamConfigFactory"; @@ -161,8 +180,12 @@ public Optional getDefaultSystem() { } public int getContainerCount() { + Optional autoscalingContainerCountValue = Optional.ofNullable(get(JOB_AUTOSIZING_CONTAINER_COUNT)); Optional jobContainerCountValue = Optional.ofNullable(get(JOB_CONTAINER_COUNT)); - if (jobContainerCountValue.isPresent()) { + + if (getAutosizingEnabled() && autoscalingContainerCountValue.isPresent()) { + return Integer.parseInt(autoscalingContainerCountValue.get()); + } else if (jobContainerCountValue.isPresent()) { return Integer.parseInt(jobContainerCountValue.get()); } else { // To maintain backwards compatibility, honor yarn.container.count for now. @@ -195,20 +218,20 @@ public int getMonitorPartitionChangeFrequency() { public Map getMonitorRegexPatternMap(String rewritersList) { Map inputRegexesToMonitor = new HashMap<>(); Stream.of(rewritersList.split(",")).forEach(rewriterName -> { - Optional rewriterSystem = getRegexResolvedSystem(rewriterName); - Optional rewriterRegex = getRegexResolvedStreams(rewriterName); - if (rewriterSystem.isPresent() && rewriterRegex.isPresent()) { - Pattern newPatternForSystem; - Pattern existingPatternForSystem = inputRegexesToMonitor.get(rewriterSystem.get()); - if (existingPatternForSystem == null) { - newPatternForSystem = Pattern.compile(rewriterRegex.get()); - } else { - newPatternForSystem = - Pattern.compile(String.join("|", existingPatternForSystem.pattern(), rewriterRegex.get())); - } - inputRegexesToMonitor.put(rewriterSystem.get(), newPatternForSystem); + Optional rewriterSystem = getRegexResolvedSystem(rewriterName); + Optional rewriterRegex = getRegexResolvedStreams(rewriterName); + if (rewriterSystem.isPresent() && rewriterRegex.isPresent()) { + Pattern newPatternForSystem; + Pattern existingPatternForSystem = inputRegexesToMonitor.get(rewriterSystem.get()); + if (existingPatternForSystem == null) { + newPatternForSystem = Pattern.compile(rewriterRegex.get()); + } else { + newPatternForSystem = + Pattern.compile(String.join("|", existingPatternForSystem.pattern(), rewriterRegex.get())); } - }); + inputRegexesToMonitor.put(rewriterSystem.get(), newPatternForSystem); + } + }); return inputRegexesToMonitor; } @@ -282,7 +305,13 @@ public String getSSPMatcherConfigJobFactoryRegex() { } public int getThreadPoolSize() { - return getInt(JOB_CONTAINER_THREAD_POOL_SIZE, 0); + Optional autoscalingContainerThreadPoolSize = Optional.ofNullable(get( + JOB_AUTOSIZING_CONTAINER_THREAD_POOL_SIZE)); + if (getAutosizingEnabled() && autoscalingContainerThreadPoolSize.isPresent()) { + return Integer.parseInt(autoscalingContainerThreadPoolSize.get()); + } else { + return getInt(JOB_CONTAINER_THREAD_POOL_SIZE, 0); + } } public int getDebounceTimeMs() { @@ -305,6 +334,23 @@ public boolean getDiagnosticsEnabled() { return getBoolean(JOB_DIAGNOSTICS_ENABLED, false); } + public boolean getAutosizingEnabled() { + return getBoolean(JOB_AUTOSIZING_ENABLED, false); + } + + public boolean isAutosizingConfig(String configParam) { + switch (configParam) { + case JOB_AUTOSIZING_CONTAINER_COUNT: + case JOB_AUTOSIZING_CONTAINER_MAX_CORES: + case JOB_AUTOSIZING_CONTAINER_MAX_HEAP_MB: + case JOB_AUTOSIZING_CONTAINER_MEMORY_MB: + case JOB_AUTOSIZING_CONTAINER_THREAD_POOL_SIZE: + return true; + default: + return false; + } + } + public boolean getJMXEnabled() { return getBoolean(JOB_JMX_ENABLED, true); } diff --git a/samza-core/src/main/scala/org/apache/samza/config/ShellCommandConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/ShellCommandConfig.scala index 702445fdc8..f1f9688e8d 100644 --- a/samza-core/src/main/scala/org/apache/samza/config/ShellCommandConfig.scala +++ b/samza-core/src/main/scala/org/apache/samza/config/ShellCommandConfig.scala @@ -25,6 +25,11 @@ object ShellCommandConfig { */ val ENV_COORDINATOR_SYSTEM_CONFIG = "SAMZA_COORDINATOR_SYSTEM_CONFIG" + /** + * This enviorment variable is used to pass configs provided during job submission. + */ + val ENV_SUBMISSION_CONFIG = "SAMZA_SUBMISSION_CONFIG" + /** * The ID for a container. This is a string representation that is unique to the runtime environment. */ @@ -46,8 +51,8 @@ object ShellCommandConfig { val ENV_JAVA_HOME = "JAVA_HOME" /** - * The ID assigned to the container by the execution environment (eg: YARN Container Id) - */ + * The ID assigned to the container by the execution environment (eg: YARN Container Id) + */ val ENV_EXECUTION_ENV_CONTAINER_ID = "EXECUTION_ENV_CONTAINER_ID" /** From 78b61506d8ae31c7551d54d30eebcb7fbe1d6cf6 Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Mon, 6 Jan 2020 11:26:16 -0800 Subject: [PATCH 21/49] Update javadoc --- .../org/apache/samza/config/JobConfig.java | 42 ++----------------- .../samza/config/ShellCommandConfig.scala | 2 +- 2 files changed, 4 insertions(+), 40 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/config/JobConfig.java b/samza-core/src/main/java/org/apache/samza/config/JobConfig.java index a64cc9f839..bf32b41724 100644 --- a/samza-core/src/main/java/org/apache/samza/config/JobConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/JobConfig.java @@ -53,6 +53,7 @@ public class JobConfig extends MapConfig { public static final String CONFIG_LOADER_FACTORY = "job.config.loader.factory"; /** * job.config.loader.properties prefix wraps properties needed for {@link ConfigLoader} to loader config. + * e.g. {@link PropertiesConfigLoaderFactory} will read job.config.loader.properties.path to locate the config file location */ public static final String CONFIG_LOADER_PROPERTIES_PREFIX = "job.config.loader.properties."; @@ -134,16 +135,6 @@ public class JobConfig extends MapConfig { public static final String CONTAINER_METADATA_FILENAME_FORMAT = "%s.metadata"; // Filename: .metadata public static final String CONTAINER_METADATA_DIRECTORY_SYS_PROPERTY = "samza.log.dir"; - - // Auto-sizing related configs tthat ake precedence over respective sizing confings job.container.count, etc, - // *only* when job.autosizing.enabled is true. Otherwise current behavior is maintained. - public static final String JOB_AUTOSIZING_ENABLED = "job.autosizing.enabled"; - public static final String JOB_AUTOSIZING_CONTAINER_COUNT = "job.autosizing.container.count"; - public static final String JOB_AUTOSIZING_CONTAINER_THREAD_POOL_SIZE = "job.autosizing.container.thread.pool.size"; - public static final String JOB_AUTOSIZING_CONTAINER_MAX_HEAP_MB = "job.autosizing.container.maxheap.mb"; - public static final String JOB_AUTOSIZING_CONTAINER_MEMORY_MB = "job.autosizing.container.memory.mb"; - public static final String JOB_AUTOSIZING_CONTAINER_MAX_CORES = "job.autosizing.container.cpu.cores"; - public static final String COORDINATOR_STREAM_FACTORY = "job.coordinatorstream.config.factory"; public static final String DEFAULT_COORDINATOR_STREAM_CONFIG_FACTORY = "org.apache.samza.util.DefaultCoordinatorStreamConfigFactory"; @@ -180,12 +171,8 @@ public Optional getDefaultSystem() { } public int getContainerCount() { - Optional autoscalingContainerCountValue = Optional.ofNullable(get(JOB_AUTOSIZING_CONTAINER_COUNT)); Optional jobContainerCountValue = Optional.ofNullable(get(JOB_CONTAINER_COUNT)); - - if (getAutosizingEnabled() && autoscalingContainerCountValue.isPresent()) { - return Integer.parseInt(autoscalingContainerCountValue.get()); - } else if (jobContainerCountValue.isPresent()) { + if (jobContainerCountValue.isPresent()) { return Integer.parseInt(jobContainerCountValue.get()); } else { // To maintain backwards compatibility, honor yarn.container.count for now. @@ -305,13 +292,7 @@ public String getSSPMatcherConfigJobFactoryRegex() { } public int getThreadPoolSize() { - Optional autoscalingContainerThreadPoolSize = Optional.ofNullable(get( - JOB_AUTOSIZING_CONTAINER_THREAD_POOL_SIZE)); - if (getAutosizingEnabled() && autoscalingContainerThreadPoolSize.isPresent()) { - return Integer.parseInt(autoscalingContainerThreadPoolSize.get()); - } else { - return getInt(JOB_CONTAINER_THREAD_POOL_SIZE, 0); - } + return getInt(JOB_CONTAINER_THREAD_POOL_SIZE, 0); } public int getDebounceTimeMs() { @@ -334,23 +315,6 @@ public boolean getDiagnosticsEnabled() { return getBoolean(JOB_DIAGNOSTICS_ENABLED, false); } - public boolean getAutosizingEnabled() { - return getBoolean(JOB_AUTOSIZING_ENABLED, false); - } - - public boolean isAutosizingConfig(String configParam) { - switch (configParam) { - case JOB_AUTOSIZING_CONTAINER_COUNT: - case JOB_AUTOSIZING_CONTAINER_MAX_CORES: - case JOB_AUTOSIZING_CONTAINER_MAX_HEAP_MB: - case JOB_AUTOSIZING_CONTAINER_MEMORY_MB: - case JOB_AUTOSIZING_CONTAINER_THREAD_POOL_SIZE: - return true; - default: - return false; - } - } - public boolean getJMXEnabled() { return getBoolean(JOB_JMX_ENABLED, true); } diff --git a/samza-core/src/main/scala/org/apache/samza/config/ShellCommandConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/ShellCommandConfig.scala index f1f9688e8d..09421b72b5 100644 --- a/samza-core/src/main/scala/org/apache/samza/config/ShellCommandConfig.scala +++ b/samza-core/src/main/scala/org/apache/samza/config/ShellCommandConfig.scala @@ -26,7 +26,7 @@ object ShellCommandConfig { val ENV_COORDINATOR_SYSTEM_CONFIG = "SAMZA_COORDINATOR_SYSTEM_CONFIG" /** - * This enviorment variable is used to pass configs provided during job submission. + * This enviorment variable is used to pass a JSON serialized map of configs provided during job submission. */ val ENV_SUBMISSION_CONFIG = "SAMZA_SUBMISSION_CONFIG" From fdee896eaed21b34d2d74d64fcd61835014fd739 Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Mon, 6 Jan 2020 15:56:04 -0800 Subject: [PATCH 22/49] Add getConfigLoaderFactory() for easy access to config loader factory. --- .../src/main/java/org/apache/samza/config/JobConfig.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/samza-core/src/main/java/org/apache/samza/config/JobConfig.java b/samza-core/src/main/java/org/apache/samza/config/JobConfig.java index bf32b41724..d7e48b8179 100644 --- a/samza-core/src/main/java/org/apache/samza/config/JobConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/JobConfig.java @@ -358,4 +358,11 @@ public String getCoordinatorStreamFactory() { return get(COORDINATOR_STREAM_FACTORY, DEFAULT_COORDINATOR_STREAM_CONFIG_FACTORY); } + /** + * Get config loader factory according to the configs + * @return full qualified name of {@link ConfigLoaderFactory} + */ + public String getConfigLoaderFactory() { + return get(CONFIG_LOADER_FACTORY); + } } \ No newline at end of file From 3e72812f52922acb6ae956b242dd50a18dfeb42e Mon Sep 17 00:00:00 2001 From: Abhishek Shivanna Date: Fri, 20 Dec 2019 14:35:07 -0800 Subject: [PATCH 23/49] SAMZA-2423: Heartbeat failure causes incorrect container shutdown (#1240) Populate container exception from the listener only if it is null --- .../java/org/apache/samza/runtime/ContainerLaunchUtil.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/samza-core/src/main/java/org/apache/samza/runtime/ContainerLaunchUtil.java b/samza-core/src/main/java/org/apache/samza/runtime/ContainerLaunchUtil.java index 47f7398562..54cb29898f 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/ContainerLaunchUtil.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/ContainerLaunchUtil.java @@ -142,7 +142,12 @@ private static void run( heartbeatMonitor.stop(); } - containerRunnerException = listener.getContainerException(); + // Check to see if the HeartbeatMonitor has set an exception before + // overriding the value with what the listener returns + if (containerRunnerException == null) { + containerRunnerException = listener.getContainerException(); + } + if (containerRunnerException != null) { log.error("Container stopped with Exception. Exiting process now.", containerRunnerException); System.exit(1); From f1e0a35c569b9204052d2758b04ce9219dc808da Mon Sep 17 00:00:00 2001 From: Aditya Toomula Date: Thu, 2 Jan 2020 09:01:16 -0800 Subject: [PATCH 24/49] SAMZA-2425: Add support for sub-query in joins --- .../samza/sql/translator/JoinTranslator.java | 26 +++++++++++----- .../test/samzasql/TestSamzaSqlEndToEnd.java | 31 +++++++++++++++++++ 2 files changed, 50 insertions(+), 7 deletions(-) diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/JoinTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/JoinTranslator.java index 0d625bc757..635d0bace5 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/translator/JoinTranslator.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/JoinTranslator.java @@ -28,6 +28,8 @@ import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.logical.LogicalFilter; import org.apache.calcite.rel.logical.LogicalJoin; import org.apache.calcite.rel.logical.LogicalProject; import org.apache.calcite.rex.RexCall; @@ -292,17 +294,27 @@ private String dumpRelPlanForNode(RelNode relNode) { SqlExplainLevel.EXPPLAN_ATTRIBUTES); } - private SqlIOConfig resolveSourceConfigForTable(RelNode relNode, TranslatorContext context) { + private SqlIOConfig resolveSQlIOForTable(RelNode relNode, TranslatorContext context) { + // Let's recursively get to the TableScan node to identify IO for the table. if (relNode instanceof LogicalProject) { - return resolveSourceConfigForTable(((LogicalProject) relNode).getInput(), context); + return resolveSQlIOForTable(((LogicalProject) relNode).getInput(), context); } - // We are returning the sourceConfig for the table as null when the table is in another join rather than an output - // table, that's because the output of stream-table join is considered a stream. - if (relNode.getInputs().size() > 1) { + if (relNode instanceof LogicalFilter) { + return resolveSQlIOForTable(((LogicalFilter) relNode).getInput(), context); + } + + // We return null for table IO as the table seems to be involved in another join. The output of stream-table join + // is considered a stream. Hence, we return null for the table. + if (relNode instanceof LogicalJoin && relNode.getInputs().size() > 1) { return null; } + if (!(relNode instanceof TableScan)) { + throw new SamzaException(String.format("Unsupported query. relNode %s is not of type TableScan.", + relNode.toString())); + } + String sourceName = SqlIOConfig.getSourceFromSourceParts(relNode.getTable().getQualifiedName()); SqlIOConfig sourceConfig = context.getExecutionContext().getSamzaSqlApplicationConfig().getInputSystemStreamConfigBySource().get(sourceName); @@ -320,7 +332,7 @@ private JoinInputNode.InputType getInputType(RelNode relNode, TranslatorContext // The join key(s) for the table could be an udf in which case the relNode would be LogicalProject. if (relNode instanceof EnumerableTableScan || relNode instanceof LogicalProject) { - SqlIOConfig sourceTableConfig = resolveSourceConfigForTable(relNode, context); + SqlIOConfig sourceTableConfig = resolveSQlIOForTable(relNode, context); if (sourceTableConfig == null || !sourceTableConfig.getTableDescriptor().isPresent()) { return JoinInputNode.InputType.STREAM; } else if (sourceTableConfig.getTableDescriptor().get() instanceof RemoteTableDescriptor || @@ -336,7 +348,7 @@ private JoinInputNode.InputType getInputType(RelNode relNode, TranslatorContext private Table getTable(JoinInputNode tableNode, TranslatorContext context) { - SqlIOConfig sourceTableConfig = resolveSourceConfigForTable(tableNode.getRelNode(), context); + SqlIOConfig sourceTableConfig = resolveSQlIOForTable(tableNode.getRelNode(), context); if (sourceTableConfig == null || !sourceTableConfig.getTableDescriptor().isPresent()) { String errMsg = "Failed to resolve table source in join operation: node=" + tableNode.getRelNode(); diff --git a/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java b/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java index 52fba62223..26ba0eca98 100644 --- a/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java +++ b/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java @@ -760,6 +760,37 @@ public void testEndToEndStreamTableInnerJoinWithPrimaryKey() throws Exception { Assert.assertEquals(expectedOutMessages, outMessages); } + @Test + public void testEndToEndStreamTableJoinWithSubQuery() throws Exception { + int numMessages = 20; + + TestAvroSystemFactory.messages.clear(); + Map staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(numMessages); + String sql = + "Insert into testavro.enrichedPageViewTopic" + + " select p.name as profileName, pv.pageKey as pageKey, p.address as profileAddress, coalesce(null, 'N/A') as companyName" + + " from (SELECT * FROM (SELECT * from testavro.PAGEVIEW pv1 where pv1.profileId=0) as pv2) as pv" + + " join testavro.PROFILE.`$table` as p" + + " on p.id = pv.profileId"; + + List sqlStmts = Arrays.asList(sql); + staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts)); + + Config config = new MapConfig(staticConfigs); + new SamzaSqlValidator(config).validate(sqlStmts); + + runApplication(config); + + List outMessages = TestAvroSystemFactory.messages.stream() + .map(x -> ((GenericRecord) x.getMessage()).get("pageKey").toString() + "," + + (((GenericRecord) x.getMessage()).get("profileName") == null ? "null" : + ((GenericRecord) x.getMessage()).get("profileName").toString())) + .collect(Collectors.toList()); + Assert.assertEquals(1, outMessages.size()); + List expectedOutMessages = TestAvroSystemFactory.getPageKeyProfileNameJoin(1); + Assert.assertEquals(expectedOutMessages, outMessages); + } + @Test public void testEndToEndStreamTableInnerJoinWithUdf() throws Exception { int numMessages = 20; From 77499b78bee8f7ccc2b70680d32a0df2514d5eb5 Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Wed, 8 Jan 2020 10:17:55 -0800 Subject: [PATCH 25/49] Revert indentions change --- .../org/apache/samza/config/JobConfig.java | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/config/JobConfig.java b/samza-core/src/main/java/org/apache/samza/config/JobConfig.java index d7e48b8179..8aff7a5434 100644 --- a/samza-core/src/main/java/org/apache/samza/config/JobConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/JobConfig.java @@ -205,20 +205,20 @@ public int getMonitorPartitionChangeFrequency() { public Map getMonitorRegexPatternMap(String rewritersList) { Map inputRegexesToMonitor = new HashMap<>(); Stream.of(rewritersList.split(",")).forEach(rewriterName -> { - Optional rewriterSystem = getRegexResolvedSystem(rewriterName); - Optional rewriterRegex = getRegexResolvedStreams(rewriterName); - if (rewriterSystem.isPresent() && rewriterRegex.isPresent()) { - Pattern newPatternForSystem; - Pattern existingPatternForSystem = inputRegexesToMonitor.get(rewriterSystem.get()); - if (existingPatternForSystem == null) { - newPatternForSystem = Pattern.compile(rewriterRegex.get()); - } else { - newPatternForSystem = - Pattern.compile(String.join("|", existingPatternForSystem.pattern(), rewriterRegex.get())); + Optional rewriterSystem = getRegexResolvedSystem(rewriterName); + Optional rewriterRegex = getRegexResolvedStreams(rewriterName); + if (rewriterSystem.isPresent() && rewriterRegex.isPresent()) { + Pattern newPatternForSystem; + Pattern existingPatternForSystem = inputRegexesToMonitor.get(rewriterSystem.get()); + if (existingPatternForSystem == null) { + newPatternForSystem = Pattern.compile(rewriterRegex.get()); + } else { + newPatternForSystem = + Pattern.compile(String.join("|", existingPatternForSystem.pattern(), rewriterRegex.get())); + } + inputRegexesToMonitor.put(rewriterSystem.get(), newPatternForSystem); } - inputRegexesToMonitor.put(rewriterSystem.get(), newPatternForSystem); - } - }); + }); return inputRegexesToMonitor; } From bfb4d435c9e73b2de153754f0aad8e0aa14fb45e Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Wed, 8 Jan 2020 10:45:54 -0800 Subject: [PATCH 26/49] Fix typo and switch to Optional for getConfigLoaderFactory --- .../src/main/java/org/apache/samza/config/JobConfig.java | 4 ++-- .../scala/org/apache/samza/config/ShellCommandConfig.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/config/JobConfig.java b/samza-core/src/main/java/org/apache/samza/config/JobConfig.java index 8aff7a5434..9dfb63885f 100644 --- a/samza-core/src/main/java/org/apache/samza/config/JobConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/JobConfig.java @@ -362,7 +362,7 @@ public String getCoordinatorStreamFactory() { * Get config loader factory according to the configs * @return full qualified name of {@link ConfigLoaderFactory} */ - public String getConfigLoaderFactory() { - return get(CONFIG_LOADER_FACTORY); + public Optional getConfigLoaderFactory() { + return Optional.ofNullable(get(CONFIG_LOADER_FACTORY)); } } \ No newline at end of file diff --git a/samza-core/src/main/scala/org/apache/samza/config/ShellCommandConfig.scala b/samza-core/src/main/scala/org/apache/samza/config/ShellCommandConfig.scala index 09421b72b5..bf524d6979 100644 --- a/samza-core/src/main/scala/org/apache/samza/config/ShellCommandConfig.scala +++ b/samza-core/src/main/scala/org/apache/samza/config/ShellCommandConfig.scala @@ -26,7 +26,7 @@ object ShellCommandConfig { val ENV_COORDINATOR_SYSTEM_CONFIG = "SAMZA_COORDINATOR_SYSTEM_CONFIG" /** - * This enviorment variable is used to pass a JSON serialized map of configs provided during job submission. + * This environment variable is used to pass a JSON serialized map of configs provided during job submission. */ val ENV_SUBMISSION_CONFIG = "SAMZA_SUBMISSION_CONFIG" From abbcf862c46c9c5fa76e07a677d01d4fd19213fd Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Tue, 17 Dec 2019 17:07:10 -0800 Subject: [PATCH 27/49] SAMZA-2406: Add ConfigLoader, ConfigLoaderFactory interface and default PropertiesConfigLoader as well as PropertiesConfigLoaderFactory implementations. Add ConfigLoader, ConfigLoaderFactory interface and default PropertiesConfigLoader as well as PropertiesConfigLoaderFactory implementations, which will be used on AM to fetch configs. It will also be used on LocalApplicationRunner to fetch configs as well. The existing ConfigFactory interface will be deprecated and removed as we do not fetch configs on start up script run-app.sh anymore. Design: https://cwiki.apache.org/confluence/display/SAMZA/SEP-23%3A+Simplify+Job+Runner Changes: 1. Add ConfigLoader interface which will be used in job coordinator to retrieve config from. 2. Add ConfigLoaderFactory interface which will be used to create an instance of ConfigLoader. 3. Add PropertiesConfigLoader which is a default implementation of ConfigLoader that reads a local properties file given the file path. 4. Add PropertiesConfigLoaderFactory which is a default implementation of ConfigLoaderFactory that reads properties file path from start up config and creates a PropertiesConfigLoader with the provided path. API Changes: N/A. This is part of a series PRs, detailed information will be provided in the last/main PR. Upgrade Instructions: N/A. This is part of a series PRs, detailed information will be provided in the last/main PR. Usage Instructions: N/A. This is part of a series PRs, detailed information will be provided in the last/main PR. Tests: Add TestPropertiesConfigLoader to unit test PropertiesConfigLoader. --- .../org/apache/samza/config/ConfigLoader.java | 27 ++++++++ .../samza/config/ConfigLoaderFactory.java | 33 ++++++++++ .../org/apache/samza/config/MapConfig.java | 12 ++++ .../loaders/PropertiesConfigLoader.java | 61 +++++++++++++++++++ .../PropertiesConfigLoaderFactory.java | 22 +++++++ .../loaders/TestPropertiesConfigLoader.java | 33 ++++++++++ 6 files changed, 188 insertions(+) create mode 100644 samza-api/src/main/java/org/apache/samza/config/ConfigLoader.java create mode 100644 samza-api/src/main/java/org/apache/samza/config/ConfigLoaderFactory.java create mode 100644 samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoader.java create mode 100644 samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoaderFactory.java create mode 100644 samza-core/src/test/java/org/apache/samza/config/loaders/TestPropertiesConfigLoader.java diff --git a/samza-api/src/main/java/org/apache/samza/config/ConfigLoader.java b/samza-api/src/main/java/org/apache/samza/config/ConfigLoader.java new file mode 100644 index 0000000000..c607123a6a --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/config/ConfigLoader.java @@ -0,0 +1,27 @@ +/* + * 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.config; + +/** + * The primary means of fetching {@link org.apache.samza.config.Config} on Application Master during start up. + */ +public interface ConfigLoader { + Config getConfig(); +} diff --git a/samza-api/src/main/java/org/apache/samza/config/ConfigLoaderFactory.java b/samza-api/src/main/java/org/apache/samza/config/ConfigLoaderFactory.java new file mode 100644 index 0000000000..32f4295446 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/config/ConfigLoaderFactory.java @@ -0,0 +1,33 @@ +/* + * 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.config; + +/** + * The factory for an {@link ConfigLoader} instance to loader job config. + */ +public interface ConfigLoaderFactory { + /** + * Get an instance of {@link ConfigLoader} + * + * @param config start up config which specifies properties needed for the loader to load config. + * @return a config loader instance. + */ + ConfigLoader getLoader(Config config); +} diff --git a/samza-api/src/main/java/org/apache/samza/config/MapConfig.java b/samza-api/src/main/java/org/apache/samza/config/MapConfig.java index 5af2535d6d..b65b8fb1f3 100644 --- a/samza-api/src/main/java/org/apache/samza/config/MapConfig.java +++ b/samza-api/src/main/java/org/apache/samza/config/MapConfig.java @@ -25,6 +25,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Set; /** @@ -41,6 +42,17 @@ public MapConfig(Map map) { this(Collections.singletonList(map)); } + /** + * Build a {@link MapConfig} from {@link Properties} + * + * @param properties to build MapConfig + */ + public MapConfig(Properties properties) { + this.map = new HashMap<>(); + // Per Properties JavaDoc, all its keys and values are of type String + properties.forEach((key, value) -> this.map.put(key.toString(), value.toString())); + } + public MapConfig(List> maps) { this.map = new HashMap<>(); for (Map m: maps) { diff --git a/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoader.java b/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoader.java new file mode 100644 index 0000000000..115e125967 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoader.java @@ -0,0 +1,61 @@ +/* + * 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.config.loaders; + +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Properties; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.config.ConfigLoader; +import org.apache.samza.config.MapConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static java.util.Objects.requireNonNull; + + +public class PropertiesConfigLoader implements ConfigLoader { + private static final Logger LOG = LoggerFactory.getLogger(PropertiesConfigLoader.class); + + private final String path; + + public PropertiesConfigLoader(String path) { + this.path = requireNonNull(path); + } + + @Override + public Config getConfig() { + try { + InputStream in = new FileInputStream(path); + Properties props = new Properties(); + + props.load(in); + in.close(); + + LOG.debug("got config {} from config {}", props, path); + + return new MapConfig(props); + } catch (IOException e) { + throw new SamzaException("Failed to read from"); + } + } +} \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoaderFactory.java b/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoaderFactory.java new file mode 100644 index 0000000000..99db34269b --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoaderFactory.java @@ -0,0 +1,22 @@ +package org.apache.samza.config.loaders; + +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.config.ConfigLoader; +import org.apache.samza.config.ConfigLoaderFactory; + + +public class PropertiesConfigLoaderFactory implements ConfigLoaderFactory { + private static final String KEY = "path"; + + @Override + public ConfigLoader getLoader(Config config) { + String path = config.get(KEY); + + if (path == null) { + throw new SamzaException("path is required to read config from properties file"); + } + + return new PropertiesConfigLoader(path); + } +} diff --git a/samza-core/src/test/java/org/apache/samza/config/loaders/TestPropertiesConfigLoader.java b/samza-core/src/test/java/org/apache/samza/config/loaders/TestPropertiesConfigLoader.java new file mode 100644 index 0000000000..107a2050c0 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/config/loaders/TestPropertiesConfigLoader.java @@ -0,0 +1,33 @@ +package org.apache.samza.config.loaders; + +import java.util.Collections; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.config.ConfigLoader; +import org.apache.samza.config.MapConfig; +import org.junit.Test; + +import static org.junit.Assert.*; + + +public class TestPropertiesConfigLoader { + @Test + public void testCanReadPropertiesConfigFiles() { + ConfigLoader loader = new PropertiesConfigLoaderFactory().getLoader( + new MapConfig(Collections.singletonMap("path", getClass().getResource("/test.properties").getPath()))); + + Config config = loader.getConfig(); + assertEquals("bar", config.get("foo")); + } + + @Test + public void testCanNotReadWithoutPath() { + try { + ConfigLoader loader = new PropertiesConfigLoaderFactory().getLoader(new MapConfig()); + loader.getConfig(); + fail("should have gotten a samza exception"); + } catch (SamzaException e) { + // Do nothing + } + } +} \ No newline at end of file From 6d841d2a8ec9caf35210fe1b59685ac8796df9a7 Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Tue, 17 Dec 2019 17:30:06 -0800 Subject: [PATCH 28/49] Add license info --- .../PropertiesConfigLoaderFactory.java | 19 +++++++++++++++++++ .../loaders/TestPropertiesConfigLoader.java | 19 +++++++++++++++++++ 2 files changed, 38 insertions(+) diff --git a/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoaderFactory.java b/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoaderFactory.java index 99db34269b..213e320920 100644 --- a/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoaderFactory.java +++ b/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoaderFactory.java @@ -1,3 +1,22 @@ +/* + * 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.config.loaders; import org.apache.samza.SamzaException; diff --git a/samza-core/src/test/java/org/apache/samza/config/loaders/TestPropertiesConfigLoader.java b/samza-core/src/test/java/org/apache/samza/config/loaders/TestPropertiesConfigLoader.java index 107a2050c0..e9f1bad290 100644 --- a/samza-core/src/test/java/org/apache/samza/config/loaders/TestPropertiesConfigLoader.java +++ b/samza-core/src/test/java/org/apache/samza/config/loaders/TestPropertiesConfigLoader.java @@ -1,3 +1,22 @@ +/* + * 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.config.loaders; import java.util.Collections; From 30875acbb18e8683cbb494bc6d751bc8aed2b688 Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Mon, 6 Jan 2020 14:36:44 -0800 Subject: [PATCH 29/49] Update javadoc --- .../org/apache/samza/config/ConfigLoader.java | 2 +- .../java/org/apache/samza/config/MapConfig.java | 11 ----------- .../config/loaders/PropertiesConfigLoader.java | 15 ++++++++++++--- .../loaders/PropertiesConfigLoaderFactory.java | 4 ++++ .../loaders/TestPropertiesConfigLoader.java | 13 ++++--------- 5 files changed, 21 insertions(+), 24 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/config/ConfigLoader.java b/samza-api/src/main/java/org/apache/samza/config/ConfigLoader.java index c607123a6a..a392e78e9f 100644 --- a/samza-api/src/main/java/org/apache/samza/config/ConfigLoader.java +++ b/samza-api/src/main/java/org/apache/samza/config/ConfigLoader.java @@ -20,7 +20,7 @@ package org.apache.samza.config; /** - * The primary means of fetching {@link org.apache.samza.config.Config} on Application Master during start up. + * The primary means of fetching full job {@link org.apache.samza.config.Config} on cluster based job coordinator during start up. */ public interface ConfigLoader { Config getConfig(); diff --git a/samza-api/src/main/java/org/apache/samza/config/MapConfig.java b/samza-api/src/main/java/org/apache/samza/config/MapConfig.java index b65b8fb1f3..a51aff753f 100644 --- a/samza-api/src/main/java/org/apache/samza/config/MapConfig.java +++ b/samza-api/src/main/java/org/apache/samza/config/MapConfig.java @@ -42,17 +42,6 @@ public MapConfig(Map map) { this(Collections.singletonList(map)); } - /** - * Build a {@link MapConfig} from {@link Properties} - * - * @param properties to build MapConfig - */ - public MapConfig(Properties properties) { - this.map = new HashMap<>(); - // Per Properties JavaDoc, all its keys and values are of type String - properties.forEach((key, value) -> this.map.put(key.toString(), value.toString())); - } - public MapConfig(List> maps) { this.map = new HashMap<>(); for (Map m: maps) { diff --git a/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoader.java b/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoader.java index 115e125967..5cce8da4fd 100644 --- a/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoader.java +++ b/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoader.java @@ -22,6 +22,8 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; +import java.util.HashMap; +import java.util.Map; import java.util.Properties; import org.apache.samza.SamzaException; import org.apache.samza.config.Config; @@ -33,6 +35,9 @@ import static java.util.Objects.requireNonNull; +/** + * ConfigLoader to load full job configs from a local properties file given its path. + */ public class PropertiesConfigLoader implements ConfigLoader { private static final Logger LOG = LoggerFactory.getLogger(PropertiesConfigLoader.class); @@ -51,11 +56,15 @@ public Config getConfig() { props.load(in); in.close(); - LOG.debug("got config {} from config {}", props, path); + LOG.debug("got config {} from path {}", props, path); + + Map config = new HashMap<>(props.size()); + // Per Properties JavaDoc, all its keys and values are of type String + props.forEach((key, value) -> config.put(key.toString(), value.toString())); - return new MapConfig(props); + return new MapConfig(config); } catch (IOException e) { - throw new SamzaException("Failed to read from"); + throw new SamzaException("Failed to read from " + path); } } } \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoaderFactory.java b/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoaderFactory.java index 213e320920..bfbdc5bac9 100644 --- a/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoaderFactory.java +++ b/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoaderFactory.java @@ -25,6 +25,10 @@ import org.apache.samza.config.ConfigLoaderFactory; +/** + * ConfigLoaderFactory which initialize {@link PropertiesConfigLoader} with properties file path specified in + * job.config.loader.properties.path. + */ public class PropertiesConfigLoaderFactory implements ConfigLoaderFactory { private static final String KEY = "path"; diff --git a/samza-core/src/test/java/org/apache/samza/config/loaders/TestPropertiesConfigLoader.java b/samza-core/src/test/java/org/apache/samza/config/loaders/TestPropertiesConfigLoader.java index e9f1bad290..d1a2b3439a 100644 --- a/samza-core/src/test/java/org/apache/samza/config/loaders/TestPropertiesConfigLoader.java +++ b/samza-core/src/test/java/org/apache/samza/config/loaders/TestPropertiesConfigLoader.java @@ -26,7 +26,7 @@ import org.apache.samza.config.MapConfig; import org.junit.Test; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; public class TestPropertiesConfigLoader { @@ -39,14 +39,9 @@ public void testCanReadPropertiesConfigFiles() { assertEquals("bar", config.get("foo")); } - @Test + @Test(expected = SamzaException.class) public void testCanNotReadWithoutPath() { - try { - ConfigLoader loader = new PropertiesConfigLoaderFactory().getLoader(new MapConfig()); - loader.getConfig(); - fail("should have gotten a samza exception"); - } catch (SamzaException e) { - // Do nothing - } + ConfigLoader loader = new PropertiesConfigLoaderFactory().getLoader(new MapConfig()); + loader.getConfig(); } } \ No newline at end of file From e24df2bbcc9282b3a883b39fd4185e74e4d28327 Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Mon, 6 Jan 2020 15:18:22 -0800 Subject: [PATCH 30/49] Remove unused import --- samza-api/src/main/java/org/apache/samza/config/MapConfig.java | 1 - 1 file changed, 1 deletion(-) diff --git a/samza-api/src/main/java/org/apache/samza/config/MapConfig.java b/samza-api/src/main/java/org/apache/samza/config/MapConfig.java index a51aff753f..5af2535d6d 100644 --- a/samza-api/src/main/java/org/apache/samza/config/MapConfig.java +++ b/samza-api/src/main/java/org/apache/samza/config/MapConfig.java @@ -25,7 +25,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.Set; /** From 730dbcb5b1396340edf0c1f14e9c01d1501bddc5 Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Tue, 7 Jan 2020 13:27:18 -0800 Subject: [PATCH 31/49] Update JavaDocs --- .../src/main/java/org/apache/samza/config/ConfigLoader.java | 3 ++- .../java/org/apache/samza/config/ConfigLoaderFactory.java | 4 ++-- .../samza/config/loaders/PropertiesConfigLoaderFactory.java | 4 ++-- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/config/ConfigLoader.java b/samza-api/src/main/java/org/apache/samza/config/ConfigLoader.java index a392e78e9f..a3276c2c43 100644 --- a/samza-api/src/main/java/org/apache/samza/config/ConfigLoader.java +++ b/samza-api/src/main/java/org/apache/samza/config/ConfigLoader.java @@ -20,7 +20,8 @@ package org.apache.samza.config; /** - * The primary means of fetching full job {@link org.apache.samza.config.Config} on cluster based job coordinator during start up. + * The primary means of fetching full job {@link org.apache.samza.config.Config} on + * LocalApplicationRunner and ClusterBasedJobCoordinator during start up. */ public interface ConfigLoader { Config getConfig(); diff --git a/samza-api/src/main/java/org/apache/samza/config/ConfigLoaderFactory.java b/samza-api/src/main/java/org/apache/samza/config/ConfigLoaderFactory.java index 32f4295446..acc2cf91e1 100644 --- a/samza-api/src/main/java/org/apache/samza/config/ConfigLoaderFactory.java +++ b/samza-api/src/main/java/org/apache/samza/config/ConfigLoaderFactory.java @@ -20,13 +20,13 @@ package org.apache.samza.config; /** - * The factory for an {@link ConfigLoader} instance to loader job config. + * The factory for an {@link ConfigLoader} instance to load full job config. */ public interface ConfigLoaderFactory { /** * Get an instance of {@link ConfigLoader} * - * @param config start up config which specifies properties needed for the loader to load config. + * @param config start up config which specifies properties needed for the loader to load full job config. * @return a config loader instance. */ ConfigLoader getLoader(Config config); diff --git a/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoaderFactory.java b/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoaderFactory.java index bfbdc5bac9..51da294629 100644 --- a/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoaderFactory.java +++ b/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoaderFactory.java @@ -30,11 +30,11 @@ * job.config.loader.properties.path. */ public class PropertiesConfigLoaderFactory implements ConfigLoaderFactory { - private static final String KEY = "path"; + private static final String PATH_KEY = "path"; @Override public ConfigLoader getLoader(Config config) { - String path = config.get(KEY); + String path = config.get(PATH_KEY); if (path == null) { throw new SamzaException("path is required to read config from properties file"); From bd0c537b237db702d0f4697c938cac17a0798dbb Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Tue, 7 Jan 2020 15:13:01 -0800 Subject: [PATCH 32/49] Add javadoc for PropertiesConfigLoader --- .../apache/samza/config/loaders/PropertiesConfigLoader.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoader.java b/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoader.java index 5cce8da4fd..fc6acb33ba 100644 --- a/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoader.java +++ b/samza-core/src/main/java/org/apache/samza/config/loaders/PropertiesConfigLoader.java @@ -43,6 +43,11 @@ public class PropertiesConfigLoader implements ConfigLoader { private final String path; + /** + * @param path Absolute or relative file path where the properties file locates. For example, + * in Hello Samza, + * it will be set to /__package/config/wikipedia-feed.properties + */ public PropertiesConfigLoader(String path) { this.path = requireNonNull(path); } From 677f7f5b3438da34ae4af4b58e7efd558b6430d1 Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Thu, 9 Jan 2020 11:54:30 -0800 Subject: [PATCH 33/49] Update javadoc --- .../src/main/java/org/apache/samza/config/JobConfig.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/samza-core/src/main/java/org/apache/samza/config/JobConfig.java b/samza-core/src/main/java/org/apache/samza/config/JobConfig.java index 9dfb63885f..255afd1698 100644 --- a/samza-core/src/main/java/org/apache/samza/config/JobConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/JobConfig.java @@ -53,7 +53,8 @@ public class JobConfig extends MapConfig { public static final String CONFIG_LOADER_FACTORY = "job.config.loader.factory"; /** * job.config.loader.properties prefix wraps properties needed for {@link ConfigLoader} to loader config. - * e.g. {@link PropertiesConfigLoaderFactory} will read job.config.loader.properties.path to locate the config file location + * e.g. {@link org.apache.samza.config.loaders.PropertiesConfigLoaderFactory} + * will read job.config.loader.properties.path to locate the config file location */ public static final String CONFIG_LOADER_PROPERTIES_PREFIX = "job.config.loader.properties."; From c491a132e9084aea7f695664ae4ddea52f73ff7b Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Thu, 9 Jan 2020 13:47:09 -0800 Subject: [PATCH 34/49] Move CONFIG_LOADER_PROPERTIES_PREFIX from JobConfig to ConfigLoaderFactory --- .../java/org/apache/samza/config/ConfigLoaderFactory.java | 6 ++++++ .../src/main/java/org/apache/samza/config/JobConfig.java | 6 ------ 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/config/ConfigLoaderFactory.java b/samza-api/src/main/java/org/apache/samza/config/ConfigLoaderFactory.java index acc2cf91e1..9d60a85c36 100644 --- a/samza-api/src/main/java/org/apache/samza/config/ConfigLoaderFactory.java +++ b/samza-api/src/main/java/org/apache/samza/config/ConfigLoaderFactory.java @@ -23,6 +23,12 @@ * The factory for an {@link ConfigLoader} instance to load full job config. */ public interface ConfigLoaderFactory { + /** + * job.config.loader.properties prefix wraps properties needed for {@link ConfigLoader} to loader config. + * e.g. PropertiesConfigLoaderFactory will read job.config.loader.properties.path to locate the config file location + */ + String CONFIG_LOADER_PROPERTIES_PREFIX = "job.config.loader.properties."; + /** * Get an instance of {@link ConfigLoader} * diff --git a/samza-core/src/main/java/org/apache/samza/config/JobConfig.java b/samza-core/src/main/java/org/apache/samza/config/JobConfig.java index 255afd1698..9f92487e8f 100644 --- a/samza-core/src/main/java/org/apache/samza/config/JobConfig.java +++ b/samza-core/src/main/java/org/apache/samza/config/JobConfig.java @@ -51,12 +51,6 @@ public class JobConfig extends MapConfig { * job.config.loader.factory specifies {@link ConfigLoaderFactory} to get {@link ConfigLoader} */ public static final String CONFIG_LOADER_FACTORY = "job.config.loader.factory"; - /** - * job.config.loader.properties prefix wraps properties needed for {@link ConfigLoader} to loader config. - * e.g. {@link org.apache.samza.config.loaders.PropertiesConfigLoaderFactory} - * will read job.config.loader.properties.path to locate the config file location - */ - public static final String CONFIG_LOADER_PROPERTIES_PREFIX = "job.config.loader.properties."; public static final String JOB_NAME = "job.name"; public static final String JOB_ID = "job.id"; From bda88595c267df6b292f91e10be6ea82bfa0ac44 Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Fri, 10 Jan 2020 14:06:55 -0800 Subject: [PATCH 35/49] Refactor LocalApplicationRunner (#1249) * Refactor LocalApplicationRunner Issues: LocalApplicationRunner has multiple constructors and follows different initialization logic. Changes: 1. Refactor all constructors to end up with the same private constructor. 2. Align "static final" and "final static" to "static final" 3. Group class variales based on accessibility 4. Minor fixs on inconsistent styling, return etc. Tests: Unit Tests * Revert LocalApplicationRunner(SamzaApplication app, Config config, MetadataStoreFactory metadataStoreFactory) from package private back to public --- .../samza/runtime/LocalApplicationRunner.java | 39 +++++++++++-------- 1 file changed, 23 insertions(+), 16 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java index bf6dfce2d8..9ad52b1f1d 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java @@ -76,9 +76,9 @@ public class LocalApplicationRunner implements ApplicationRunner { private static final Logger LOG = LoggerFactory.getLogger(LocalApplicationRunner.class); private static final String PROCESSOR_ID = UUID.randomUUID().toString(); - private final static String RUN_ID_METADATA_STORE = "RunIdCoordinationStore"; + private static final String RUN_ID_METADATA_STORE = "RunIdCoordinationStore"; private static final String METADATA_STORE_FACTORY_CONFIG = "metadata.store.factory"; - public final static String DEFAULT_METADATA_STORE_FACTORY = ZkMetadataStoreFactory.class.getName(); + private static final String DEFAULT_METADATA_STORE_FACTORY = ZkMetadataStoreFactory.class.getName(); private final ApplicationDescriptorImpl appDesc; private final Set> processors = ConcurrentHashMap.newKeySet(); @@ -88,9 +88,9 @@ public class LocalApplicationRunner implements ApplicationRunner { private final boolean isAppModeBatch; private final Optional coordinationUtils; private final Optional metadataStoreFactory; + private Optional runId = Optional.empty(); private Optional runIdGenerator = Optional.empty(); - private ApplicationStatus appStatus = ApplicationStatus.New; /** @@ -111,10 +111,7 @@ public LocalApplicationRunner(SamzaApplication app, Config config) { * @param metadataStoreFactory the instance of {@link MetadataStoreFactory} to read and write to coordinator stream. */ public LocalApplicationRunner(SamzaApplication app, Config config, MetadataStoreFactory metadataStoreFactory) { - this.appDesc = ApplicationDescriptorUtil.getAppDescriptor(app, config); - this.isAppModeBatch = new ApplicationConfig(config).getAppMode() == ApplicationConfig.ApplicationMode.BATCH; - this.coordinationUtils = getCoordinationUtils(config); - this.metadataStoreFactory = Optional.ofNullable(metadataStoreFactory); + this(ApplicationDescriptorUtil.getAppDescriptor(app, config), getCoordinationUtils(config), metadataStoreFactory); } /** @@ -122,12 +119,20 @@ public LocalApplicationRunner(SamzaApplication app, Config config, MetadataStore */ @VisibleForTesting LocalApplicationRunner(ApplicationDescriptorImpl appDesc, Optional coordinationUtils) { + this(appDesc, coordinationUtils, getDefaultCoordinatorStreamStoreFactory(new JobConfig(appDesc.getConfig()))); + } + + private LocalApplicationRunner( + ApplicationDescriptorImpl appDesc, + Optional coordinationUtils, + MetadataStoreFactory metadataStoreFactory) { this.appDesc = appDesc; - this.isAppModeBatch = new ApplicationConfig(appDesc.getConfig()).getAppMode() == ApplicationConfig.ApplicationMode.BATCH; + this.isAppModeBatch = isAppModeBatch(appDesc.getConfig()); this.coordinationUtils = coordinationUtils; - this.metadataStoreFactory = Optional.ofNullable(getDefaultCoordinatorStreamStoreFactory(new JobConfig(appDesc.getConfig()))); + this.metadataStoreFactory = Optional.ofNullable(metadataStoreFactory); } + @VisibleForTesting static MetadataStoreFactory getDefaultCoordinatorStreamStoreFactory(JobConfig jobConfig) { String coordinatorSystemName = jobConfig.getCoordinatorSystemNameOrNull(); JobCoordinatorConfig jobCoordinatorConfig = new JobCoordinatorConfig(jobConfig); @@ -144,8 +149,8 @@ static MetadataStoreFactory getDefaultCoordinatorStreamStoreFactory(JobConfig jo return null; } - private Optional getCoordinationUtils(Config config) { - if (!isAppModeBatch) { + private static Optional getCoordinationUtils(Config config) { + if (!isAppModeBatch(config)) { return Optional.empty(); } JobCoordinatorConfig jcConfig = new JobCoordinatorConfig(config); @@ -154,6 +159,10 @@ private Optional getCoordinationUtils(Config config) { return Optional.ofNullable(coordinationUtils); } + private static boolean isAppModeBatch(Config config) { + return new ApplicationConfig(config).getAppMode() == ApplicationConfig.ApplicationMode.BATCH; + } + /** * @return LocalJobPlanner created */ @@ -185,7 +194,7 @@ private void initializeRunId() { runIdGenerator = Optional.of(new RunIdGenerator(coordinationUtils.get(), metadataStore)); runId = runIdGenerator.flatMap(RunIdGenerator::getRunId); } catch (Exception e) { - LOG.warn("Failed to generate run id. Will continue execution without a run id. Caused by {}", e); + LOG.warn("Failed to generate run id. Will continue execution without a run id.", e); } } @@ -278,7 +287,7 @@ public boolean waitForFinish(Duration timeout) { @VisibleForTesting protected Set getProcessors() { - return processors.stream().map(sp -> sp.getLeft()).collect(Collectors.toSet()); + return processors.stream().map(Pair::getLeft).collect(Collectors.toSet()); } @VisibleForTesting @@ -340,10 +349,8 @@ StreamProcessor createStreamProcessor(Config config, ApplicationDescriptorImpl reporters.put(name, factory.getMetricsReporter(name, processorId, config))); - StreamProcessor streamProcessor = new StreamProcessor(processorId, config, reporters, taskFactory, appDesc.getApplicationContainerContextFactory(), + return new StreamProcessor(processorId, config, reporters, taskFactory, appDesc.getApplicationContainerContextFactory(), appDesc.getApplicationTaskContextFactory(), externalContextOptional, listenerFactory, null, coordinatorStreamStore); - - return streamProcessor; } /** From 6cf114a08e0c48a8ee457693e518ab294ead3486 Mon Sep 17 00:00:00 2001 From: Boris Shkolnik Date: Mon, 13 Jan 2020 15:21:58 -0800 Subject: [PATCH 36/49] Add "strict" parameter to all the log4j.xml files (#1244) --- samza-azure/src/test/resources/log4j2.xml | 4 ++-- samza-log4j2/src/test/resources/log4j2.xml | 2 +- samza-rest/src/main/resources/log4j2.xml | 4 ++-- samza-shell/src/main/resources/log4j2-console.xml | 4 ++-- samza-sql/src/test/resources/log4j2.xml | 4 ++-- samza-test/src/main/resources/log4j2.xml | 2 +- samza-tools/src/main/resources/log4j2.xml | 4 ++-- 7 files changed, 12 insertions(+), 12 deletions(-) diff --git a/samza-azure/src/test/resources/log4j2.xml b/samza-azure/src/test/resources/log4j2.xml index 21f81f70ac..db7281355f 100644 --- a/samza-azure/src/test/resources/log4j2.xml +++ b/samza-azure/src/test/resources/log4j2.xml @@ -11,7 +11,7 @@ language governing permissions and limitations under the License. --> - + @@ -29,4 +29,4 @@ - \ No newline at end of file + diff --git a/samza-log4j2/src/test/resources/log4j2.xml b/samza-log4j2/src/test/resources/log4j2.xml index 578dd5fee9..70f0b64798 100644 --- a/samza-log4j2/src/test/resources/log4j2.xml +++ b/samza-log4j2/src/test/resources/log4j2.xml @@ -11,7 +11,7 @@ language governing permissions and limitations under the License. --> - + diff --git a/samza-rest/src/main/resources/log4j2.xml b/samza-rest/src/main/resources/log4j2.xml index fdded6f16f..038d4a508a 100644 --- a/samza-rest/src/main/resources/log4j2.xml +++ b/samza-rest/src/main/resources/log4j2.xml @@ -19,7 +19,7 @@ specific language governing permissions and limitations under the License. --> - + @@ -37,4 +37,4 @@ under the License. - \ No newline at end of file + diff --git a/samza-shell/src/main/resources/log4j2-console.xml b/samza-shell/src/main/resources/log4j2-console.xml index f3dbd003f3..d3ce5d0f1b 100644 --- a/samza-shell/src/main/resources/log4j2-console.xml +++ b/samza-shell/src/main/resources/log4j2-console.xml @@ -20,7 +20,7 @@ --> - + @@ -32,4 +32,4 @@ - \ No newline at end of file + diff --git a/samza-sql/src/test/resources/log4j2.xml b/samza-sql/src/test/resources/log4j2.xml index 8cbd4b967a..ceeda36676 100644 --- a/samza-sql/src/test/resources/log4j2.xml +++ b/samza-sql/src/test/resources/log4j2.xml @@ -10,7 +10,7 @@ OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - + @@ -32,4 +32,4 @@ - \ No newline at end of file + diff --git a/samza-test/src/main/resources/log4j2.xml b/samza-test/src/main/resources/log4j2.xml index 0d60ed4e70..68e66b7d7a 100644 --- a/samza-test/src/main/resources/log4j2.xml +++ b/samza-test/src/main/resources/log4j2.xml @@ -10,7 +10,7 @@ OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - + diff --git a/samza-tools/src/main/resources/log4j2.xml b/samza-tools/src/main/resources/log4j2.xml index 03316996fe..2399bc2d9c 100644 --- a/samza-tools/src/main/resources/log4j2.xml +++ b/samza-tools/src/main/resources/log4j2.xml @@ -11,7 +11,7 @@ language governing permissions and limitations under the License. --> - + @@ -29,4 +29,4 @@ - \ No newline at end of file + From 03583a8454146eb7200d78b529f3d2af78eadf95 Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Mon, 13 Jan 2020 19:40:06 -0800 Subject: [PATCH 37/49] SAMZA-2428: Clean up unused org.apache.samza.sql.util.ConfigUtil (#1245) Issues: 1. org.apache.samza.sql.util.ConfigUtil is not used 2. it is the same name as org.apache.samza.util.ConfigUtil 3. its usage should be deprecated in favor of Config#subset(String prefix, boolean stripPrefix) Changes: delete org.apache.samza.sql.util.ConfigUtil Tests: build --- .../org/apache/samza/sql/util/ConfigUtil.java | 62 ------------------- 1 file changed, 62 deletions(-) delete mode 100644 samza-sql/src/main/java/org/apache/samza/sql/util/ConfigUtil.java diff --git a/samza-sql/src/main/java/org/apache/samza/sql/util/ConfigUtil.java b/samza-sql/src/main/java/org/apache/samza/sql/util/ConfigUtil.java deleted file mode 100644 index e87ea2ff26..0000000000 --- a/samza-sql/src/main/java/org/apache/samza/sql/util/ConfigUtil.java +++ /dev/null @@ -1,62 +0,0 @@ -/* -* 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.util; - -import java.util.Properties; - -import org.apache.commons.lang.StringUtils; - - -/** - * Utility methods to aid with config management. - */ -public class ConfigUtil { - - private ConfigUtil() { - } - - /** - * Method is used to filter just the properties with the prefix. - * @param props Full set of properties - * @param prefix Prefix of the keys that in the config that needs to be filtered out. - * @param preserveFullKey If set to true, after filtering, preserves the full key including the prefix. - * If set to false, Strips out the prefix from the key before returning. - * @return Returns the filtered set of properties matching the prefix from the input property bag. - */ - public static Properties getDomainProperties(Properties props, String prefix, boolean preserveFullKey) { - String fullPrefix; - if (StringUtils.isBlank(prefix)) { - fullPrefix = ""; // this will effectively retrieve all properties - } else { - fullPrefix = prefix.endsWith(".") ? prefix : prefix + "."; - } - Properties ret = new Properties(); - props.keySet().stream().map(String.class::cast).forEach(keyStr -> { - if (keyStr.startsWith(fullPrefix) && !keyStr.equals(fullPrefix)) { - if (preserveFullKey) { - ret.put(keyStr, props.get(keyStr)); - } else { - ret.put(keyStr.substring(fullPrefix.length()), props.get(keyStr)); - } - } - }); - return ret; - } -} From c2c73dd5d18b9344a9e26808d4882eed5728aebe Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Tue, 14 Jan 2020 12:03:24 -0800 Subject: [PATCH 38/49] Clean up unused org.apache.samza.autoscaling module (#1250) Issues: samza-autoscaling module is not used. Changes: remove the unused module. API Changes: None Upgrade Instructions: None Usage Instructions: None Tests: build --- build.gradle | 40 -- .../autoscaling/deployer/ConfigManager.java | 376 ------------------ .../samza/autoscaling/utils/YarnUtil.java | 158 -------- .../samza/autoscaling/utils/YarnUtilTest.java | 38 -- .../exampleResourceManagerOutput.json | 121 ------ .../src/main/bash/run-config-manager.sh | 25 -- settings.gradle | 1 - sonar-project.properties | 2 +- 8 files changed, 1 insertion(+), 760 deletions(-) delete mode 100644 samza-autoscaling/src/main/java/org/apache/samza/autoscaling/deployer/ConfigManager.java delete mode 100644 samza-autoscaling/src/main/java/org/apache/samza/autoscaling/utils/YarnUtil.java delete mode 100644 samza-autoscaling/src/test/java/org/apache/samza/autoscaling/utils/YarnUtilTest.java delete mode 100644 samza-autoscaling/src/test/resources/exampleResourceManagerOutput.json delete mode 100755 samza-shell/src/main/bash/run-config-manager.sh diff --git a/build.gradle b/build.gradle index c6452ac2c4..3820c8d33f 100644 --- a/build.gradle +++ b/build.gradle @@ -263,46 +263,6 @@ project(":samza-aws_$scalaSuffix") { } } - -project(":samza-autoscaling_$scalaSuffix") { - apply plugin: 'scala' - apply plugin: 'checkstyle' - - // Force scala joint compilation - sourceSets.main.scala.srcDir "src/main/java" - sourceSets.test.scala.srcDir "src/test/java" - - // Disable the Javac compiler by forcing joint compilation by scalac. This is equivalent to setting - // tasks.compileTestJava.enabled = false - sourceSets.main.java.srcDirs = [] - sourceSets.test.java.srcDirs = [] - - dependencies { - compile project(':samza-api') - compile project(":samza-core_$scalaSuffix") - compile "org.scala-lang:scala-library:$scalaVersion" - compile "org.slf4j:slf4j-api:$slf4jVersion" - compile "net.sf.jopt-simple:jopt-simple:$joptSimpleVersion" - compile "org.codehaus.jackson:jackson-mapper-asl:$jacksonVersion" - compile "org.eclipse.jetty:jetty-webapp:$jettyVersion" - compile("org.apache.hadoop:hadoop-yarn-client:$yarnVersion") { - exclude module: 'servlet-api' - } - compile("org.apache.hadoop:hadoop-common:$yarnVersion") { - exclude module: 'servlet-api' - } - compile "org.apache.httpcomponents:httpclient:$httpClientVersion" - testCompile "junit:junit:$junitVersion" - testCompile "org.mockito:mockito-core:$mockitoVersion" - testCompile "org.scalatest:scalatest_$scalaSuffix:$scalaTestVersion" - } - - checkstyle { - configFile = new File(rootDir, "checkstyle/checkstyle.xml") - toolVersion = "$checkstyleVersion" - } -} - project(":samza-elasticsearch_$scalaSuffix") { apply plugin: 'java' diff --git a/samza-autoscaling/src/main/java/org/apache/samza/autoscaling/deployer/ConfigManager.java b/samza-autoscaling/src/main/java/org/apache/samza/autoscaling/deployer/ConfigManager.java deleted file mode 100644 index 1d319d676a..0000000000 --- a/samza-autoscaling/src/main/java/org/apache/samza/autoscaling/deployer/ConfigManager.java +++ /dev/null @@ -1,376 +0,0 @@ -/* - * 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.autoscaling.deployer; - -import joptsimple.OptionSet; - -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.samza.autoscaling.utils.YarnUtil; -import org.apache.samza.config.Config; -import org.apache.samza.config.JobConfig; -import org.apache.samza.container.SamzaContainer; -import org.apache.samza.coordinator.stream.messages.CoordinatorStreamMessage; -import org.apache.samza.coordinator.stream.CoordinatorStreamSystemConsumer; -import org.apache.samza.coordinator.stream.messages.SetConfig; -import org.apache.samza.job.JobRunner; -import org.apache.samza.job.model.ContainerModel; -import org.apache.samza.metrics.MetricsRegistryMap; -import org.apache.samza.system.SystemStreamPartitionIterator; -import org.apache.samza.util.CommandLine; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; - - -/** - * This class is a separate module that runs along side with a job, and handles all config changes submitted to a job after the bootstrap of the job. - * All config changes are written to the coordinator stream using the @Link{CoordinatorStreamWriter}. - * The way this class works is that it reads all messages with type "set-config" written to the coordinator stream after - * the bootstrap of the job, and it handles the messages accordingly. - * The current configuration changes it handles are - * 1. changing the number of containers of a job - * 2. setting the server url for the first time (in order to get the JobModel). - * In order to use this class the run() method should be called to react to changes, - * or call the start(), processConfigMessages(), and stop() function instead. - * Additionally, you have to add the following configurations to the config file: - * yarn.rm.address=localhost //the ip of the resource manager in yarn - * yarn.rm.port=8088 //the port of the resource manager http server - * Additionally, the config manger will periodically poll the coordinator stream to see if there are any new messages. - * This period is set to 100 ms by default. However, it can be configured by adding the following property to the input config file. - * configManager.polling.interval=< polling interval > - */ - -public class ConfigManager { - private final CoordinatorStreamSystemConsumer coordinatorStreamConsumer; - private SystemStreamPartitionIterator coordinatorStreamIterator; - private static final Logger log = LoggerFactory.getLogger(ConfigManager.class); - private final long defaultPollingInterval = 100; - private final int defaultReadJobModelDelayMs = 100; - private final long interval; - private String coordinatorServerURL = null; - private final String jobName; - private final int jobID; - private Config config; - - private YarnUtil yarnUtil; - - private final String rmAddressOpt = "yarn.rm.address"; - private final String rmPortOpt = "yarn.rm.port"; - private final String pollingIntervalOpt = "configManager.polling.interval"; - private static final String SERVER_URL_OPT = "samza.autoscaling.server.url"; - private static final String YARN_CONTAINER_COUNT_OPT = "yarn.container.count"; - - public ConfigManager(Config config) { - - //get rm address and port - if (!config.containsKey(rmAddressOpt) || !config.containsKey(rmPortOpt)) { - throw new IllegalArgumentException("Missing config: the config file does not contain the rm host or port."); - } - String rmAddress = config.get(rmAddressOpt); - int rmPort = config.getInt(rmPortOpt); - - //get job name and id; - if (!config.containsKey(JobConfig.JOB_NAME)) { - throw new IllegalArgumentException("Missing config: the config does not contain the job name"); - } - jobName = config.get(JobConfig.JOB_NAME); - jobID = config.getInt(JobConfig.JOB_ID, 1); - - //set polling interval - if (config.containsKey(pollingIntervalOpt)) { - long pollingInterval = config.getLong(pollingIntervalOpt); - if (pollingInterval <= 0) { - throw new IllegalArgumentException("polling interval should be greater than 0"); - } - this.interval = pollingInterval; - } else { - this.interval = defaultPollingInterval; - } - - this.config = config; - this.coordinatorStreamConsumer = new CoordinatorStreamSystemConsumer(config, new MetricsRegistryMap()); - this.yarnUtil = new YarnUtil(rmAddress, rmPort); - } - - /** - * This method is an infinite loop that periodically checks if there are any new messages in the job coordinator stream, and reads them if they exist. - * Then it reacts accordingly based on the configuration that is being set. - * The method the calls the start() method to initialized the system, runs in a infinite loop, and calls the stop() method at the end to stop the consumer and the system - */ - private void run() { - start(); - try { - while (true) { - Thread.sleep(interval); - processConfigMessages(); - } - } catch (InterruptedException e) { - e.printStackTrace(); - log.warn("Got interrupt in config manager thread, so shutting down"); - Thread.currentThread().interrupt(); - } finally { - log.info("Stopping the config manager"); - stop(); - } - } - - /** - * Starts the system by starting the consumer - */ - private void start() { - register(); - coordinatorStreamConsumer.start(); - coordinatorStreamIterator = coordinatorStreamConsumer.getStartIterator(); - bootstrap(); - } - - /** - * stops the consumer making the system ready to stop - */ - private void stop() { - coordinatorStreamConsumer.stop(); - coordinatorServerURL = null; - yarnUtil.stop(); - } - - /** - * registers the consumer - */ - private void register() { - coordinatorStreamConsumer.register(); - } - - - /** - * This function will bootstrap by reading all the unread messages until the moment of calling the function, and therefore find the server url. - */ - private void bootstrap() { - List keysToProcess = new LinkedList<>(); - keysToProcess.add(SERVER_URL_OPT); - processConfigMessages(keysToProcess); - if (coordinatorServerURL == null) { - throw new IllegalStateException("coordinator server url is null, while the bootstrap has finished "); - } - log.info("Config manager bootstrapped"); - } - - /** - * notAValidEvent all the unread messages up to the time this function is called. - * This method just reads the messages, and it does not react to them or change any configuration of the system. - */ - private void skipUnreadMessages() { - processConfigMessages(Collections.emptyList()); - log.info("Config manager skipped messages"); - } - - /** - * This function reads all the messages with "set-config" type added to the coordinator stream since the last time the method was invoked - */ - private void processConfigMessages() { - List keysToProcess = Arrays.asList(YARN_CONTAINER_COUNT_OPT, SERVER_URL_OPT); - processConfigMessages(keysToProcess); - } - - /** - * This function reads all the messages with "set-config" type added to the coordinator stream since the last time the method was invoked - * - * @param keysToProcess a list of keys to process. Only messages with these keys will call their handler function, - * and other messages will be skipped. If the list is empty all messages will be skipped. - */ - @SuppressWarnings("unchecked") - private void processConfigMessages(List keysToProcess) { - if (!coordinatorStreamConsumer.hasNewMessages(coordinatorStreamIterator)) { - return; - } - if (keysToProcess == null) { - throw new IllegalArgumentException("The keys to process list is null"); - } - for (CoordinatorStreamMessage message : coordinatorStreamConsumer.getUnreadMessages(coordinatorStreamIterator, SetConfig.TYPE)) { - String key = null; - try { - SetConfig setConfigMessage = new SetConfig(message); - key = setConfigMessage.getKey(); - Map valuesMap = (Map) setConfigMessage.getMessageMap().get("values"); - String value = null; - if (valuesMap != null) { - value = valuesMap.get("value"); - } - - log.debug("Received set-config message with key: " + key + " and value: " + value); - - if (keysToProcess.contains(key)) { - if (key.equals(YARN_CONTAINER_COUNT_OPT)) { - handleYarnContainerChange(value); - } else if (key.equals(SERVER_URL_OPT)) { - handleServerURLChange(value); - } else { - log.info("Setting the " + key + " configuration is currently not supported, skipping the message"); - } - } - - //TODO: change the handlers to implement a common interface, to make them pluggable - } catch (Exception e) { - log.error("Error in reading a message, skipping message with key " + key); - } - - } - - } - - /** - * This method handle setConfig messages that want to change the url of the server the JobCoordinator has brought up. - * - * @param newServerURL the new value of the server URL - */ - private void handleServerURLChange(String newServerURL) { - this.coordinatorServerURL = newServerURL; - log.info("Server URL being set to " + newServerURL); - } - - /** - * This method handles setConfig messages that want to change the number of containers of a job - * - * @param containerCountAsString the new number of containers in a String format - */ - private void handleYarnContainerChange(String containerCountAsString) throws IOException, YarnException { - String applicationId = yarnUtil.getRunningAppId(jobName, jobID); - - int containerCount = Integer.valueOf(containerCountAsString); - - //checking the input is valid - int currentNumTask = getCurrentNumTasks(); - int currentNumContainers = getCurrentNumContainers(); - if (containerCount == currentNumContainers) { - log.error("The new number of containers is equal to the current number of containers, skipping this message"); - return; - } - if (containerCount <= 0) { - log.error("The number of containers cannot be zero or less, skipping this message"); - return; - } - - - if (containerCount > currentNumTask) { - log.error("The number of containers cannot be more than the number of task, skipping this message"); - return; - } - - - //killing the current job - log.info("Killing the current job"); - yarnUtil.killApplication(applicationId); - //reset the global variables - coordinatorServerURL = null; - - - try { - //waiting for the job to be killed - String state = yarnUtil.getApplicationState(applicationId); - Thread.sleep(1000); - int countSleep = 1; - - while (!state.equals("KILLED")) { - state = yarnUtil.getApplicationState(applicationId); - log.info("Job kill signal sent, but job not killed yet for " + applicationId + ". Sleeping for another 1000ms"); - Thread.sleep(1000); - countSleep++; - if (countSleep > 10) { - throw new IllegalStateException("Job has not been killed after 10 attempts."); - } - } - } catch (InterruptedException e) { - e.printStackTrace(); - Thread.currentThread().interrupt(); - } - - log.info("Killed the current job successfully"); - - //start the job again - log.info("Staring the job again"); - skipUnreadMessages(); - JobRunner jobRunner = new JobRunner(config); - jobRunner.run(false); - } - - - /** - * This method returns the number of tasks in the job. It works by querying the server, and getting the job model. - * Then it extracts the number of tasks from the job model - * - * @return current number of tasks in the job - */ - private int getCurrentNumTasks() { - int currentNumTasks = 0; - for (ContainerModel containerModel : SamzaContainer.readJobModel(coordinatorServerURL, defaultReadJobModelDelayMs).getContainers().values()) { - currentNumTasks += containerModel.getTasks().size(); - } - return currentNumTasks; - } - - /** - * This method returns the number of containers in the job. It works by querying the server, and getting the job model. - * Then it extracts the number of containers from the job model - * - * @return current number of containers in the job - */ - private int getCurrentNumContainers() { - return SamzaContainer.readJobModel(coordinatorServerURL, defaultReadJobModelDelayMs).getContainers().values().size(); - } - - - /** - * Gets the current value of the server URL that the job coordinator is serving the job model on. - * - * @return the current server URL. If null, it means the job has not set the server yet. - */ - public String getCoordinatorServerURL() { - return coordinatorServerURL; - } - - /** - * Main function for using the Config Manager. The main function starts a Config Manager, and reacts to all messages thereafter - * In order for this module to run, you have to add the following configurations to the config file: - * yarn.rm.address=localhost //the ip of the resource manager in yarn - * yarn.rm.port=8088 //the port of the resource manager http server - * Additionally, the config manger will periodically poll the coordinator stream to see if there are any new messages. - * This period is set to 100 ms by default. However, it can be configured by adding the following property to the input config file. - * configManager.polling.interval= < polling interval > - * To run the code use the following command: - * {path to samza deployment}/samza/bin/run-config-manager.sh --config-factory={config-factory} --config-path={path to config file of a job} - * - * @param args input arguments for running ConfigManager. - */ - public static void main(String[] args) { - CommandLine cmdline = new CommandLine(); - OptionSet options = cmdline.parser().parse(args); - Config config = cmdline.loadConfig(options); - ConfigManager configManager = new ConfigManager(config); - configManager.run(); - } - - -} diff --git a/samza-autoscaling/src/main/java/org/apache/samza/autoscaling/utils/YarnUtil.java b/samza-autoscaling/src/main/java/org/apache/samza/autoscaling/utils/YarnUtil.java deleted file mode 100644 index 7331f61c75..0000000000 --- a/samza-autoscaling/src/main/java/org/apache/samza/autoscaling/utils/YarnUtil.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * 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.autoscaling.utils; - -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.client.api.YarnClient; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.http.HttpHost; -import org.apache.http.HttpResponse; -import org.apache.http.client.methods.HttpGet; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.impl.client.HttpClientBuilder; -import org.apache.http.util.EntityUtils; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.List; -import java.util.Map; - -/** - * This is a helper class to interact with yarn. Some of the functionalities it provides are killing an application, - * getting the state of an application, getting an application id given the job name and job id. - */ -public class YarnUtil { - private static final Logger log = LoggerFactory.getLogger(YarnUtil.class); - private final CloseableHttpClient httpClient; - private final HttpHost rmServer; - private final YarnClient yarnClient; - - public YarnUtil(String rmAddress, int rmPort) { - this.httpClient = HttpClientBuilder.create().build(); - this.rmServer = new HttpHost(rmAddress, rmPort, "http"); - log.info("setting rm server to : " + rmServer); - YarnConfiguration hConfig = new YarnConfiguration(); - hConfig.set(YarnConfiguration.RM_ADDRESS, rmAddress + ":" + YarnConfiguration.DEFAULT_RM_PORT); - yarnClient = YarnClient.createYarnClient(); - yarnClient.init(hConfig); - yarnClient.start(); - } - - /** - * Queries rm for all the applications currently running and finds the application with the matching job name and id - * - * @param jobName the name of the job - * @param jobID the job id - * @return the application id of the job running in yarn. If application id is not found, it will return null. - */ - public String getRunningAppId(String jobName, int jobID) { - - try { - HttpGet getRequest = new HttpGet("/ws/v1/cluster/apps"); - HttpResponse httpResponse = httpClient.execute(rmServer, getRequest); - String applications = EntityUtils.toString(httpResponse.getEntity()); - log.debug("applications: " + applications); - - List> applicationList = parseYarnApplications(applications); - String name = jobName + "_" + jobID; - for (Map application : applicationList) { - if ("RUNNING".equals(application.get("state")) && name.equals(application.get("name")) && application.containsKey("id")) { - return application.get("id"); - } - } - } catch (NullPointerException | IOException e) { - e.printStackTrace(); - throw new IllegalStateException("there is no valid application id for the given job name and job id. job name: " + jobName + " job id: " + jobID); - } - - return null; - } - - List> parseYarnApplications(String applications) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - Map>>> yarnApplications = mapper.readValue(applications, new TypeReference>>>>() { - }); - return yarnApplications.get("apps").get("app"); - } - - /** - * This function returns the state of a given application. This state can be on of the - * {"NEW", "NEW_SAVING", "SUBMITTED", "ACCEPTED", "RUNNING", "FINISHED", "FAILED", "KILLED"} - * - * @param applicationId the application id of the application the state is being queried - * @return the state of the application which is one of the following values: {"NEW", "NEW_SAVING", "SUBMITTED", "ACCEPTED", "RUNNING", "FINISHED", "FAILED", "KILLED"} - * @throws IOException Throws IO exception - * @throws YarnException in case of errors or if YARN rejects the request due to - * access-control restrictions. - */ - public String getApplicationState(String applicationId) throws IOException, YarnException { - - return yarnClient.getApplicationReport(getApplicationIDFromString(applicationId)).getYarnApplicationState().toString(); - - } - - /** - * This function kills an application given the applicationId - * - * @param applicationId the application Id of the job to be killed - * @throws IOException Throws IO exception - * @throws YarnException in case of errors or if YARN rejects the request due to - * access-control restrictions. - */ - public void killApplication(String applicationId) throws IOException, YarnException { - - log.info("killing job with application id: " + applicationId); - - yarnClient.killApplication(getApplicationIDFromString(applicationId)); - } - - /** - * This function converts an application in form of a String into a {@link ApplicationId} - * - * @param appIDStr The application id in form of a string - * @return the application id as an instance of ApplicationId class. - */ - private ApplicationId getApplicationIDFromString(String appIDStr) { - String[] parts = appIDStr.split("_"); - if (parts.length < 3) { - throw new IllegalStateException("the application id found is not valid. application id: " + appIDStr); - } - long timestamp = Long.valueOf(parts[1]); - int id = Integer.valueOf(parts[2]); - return ApplicationId.newInstance(timestamp, id); - } - - /** - * This function stops the YarnUtil by stopping the yarn client and http client. - */ - public void stop() { - try { - httpClient.close(); - } catch (IOException e) { - log.error("HTTP Client failed to close.", e); - } - yarnClient.stop(); - } - -} diff --git a/samza-autoscaling/src/test/java/org/apache/samza/autoscaling/utils/YarnUtilTest.java b/samza-autoscaling/src/test/java/org/apache/samza/autoscaling/utils/YarnUtilTest.java deleted file mode 100644 index 7b4b74ea90..0000000000 --- a/samza-autoscaling/src/test/java/org/apache/samza/autoscaling/utils/YarnUtilTest.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.autoscaling.utils; - -import org.apache.commons.io.IOUtils; -import org.junit.Test; - -import java.io.IOException; -import java.util.List; -import java.util.Map; - -import static org.junit.Assert.assertEquals; - -public class YarnUtilTest { - - @Test - public void handleJsonArraysAsWellAsStrings() throws IOException { - YarnUtil yarnUtil = new YarnUtil("rm", 0); - List> applications = yarnUtil.parseYarnApplications(IOUtils.toString(getClass().getClassLoader().getResourceAsStream("exampleResourceManagerOutput.json"))); - assertEquals("RUNNING", applications.get(0).get("state")); - } -} diff --git a/samza-autoscaling/src/test/resources/exampleResourceManagerOutput.json b/samza-autoscaling/src/test/resources/exampleResourceManagerOutput.json deleted file mode 100644 index 9f8a02519a..0000000000 --- a/samza-autoscaling/src/test/resources/exampleResourceManagerOutput.json +++ /dev/null @@ -1,121 +0,0 @@ -{ - "apps": { - "app": [ - { - "id": "application_1459790549146_0003", - "user": "root", - "name": "protodeserializer_1", - "queue": "default", - "state": "RUNNING", - "finalStatus": "UNDEFINED", - "progress": 0, - "trackingUI": "ApplicationMaster", - "trackingUrl": "http://yarnrm:8088/proxy/application_1459790549146_0003/", - "diagnostics": "", - "clusterId": 1459790549146, - "applicationType": "Samza", - "applicationTags": "", - "startedTime": 1459792852675, - "finishedTime": 0, - "elapsedTime": 738921, - "amContainerLogs": "http://yarnnm:8042/node/containerlogs/container_1459790549146_0003_01_000001/root", - "amHostHttpAddress": "yarnnm:8042", - "allocatedMB": 1024, - "allocatedVCores": 2, - "runningContainers": 2, - "memorySeconds": 749045, - "vcoreSeconds": 1462, - "preemptedResourceMB": 0, - "preemptedResourceVCores": 0, - "numNonAMContainerPreempted": 0, - "numAMContainerPreempted": 0, - "resourceRequests": [ - { - "capability": { - "memory": 512, - "virtualCores": 1 - }, - "nodeLabelExpression": "", - "numContainers": 0, - "priority": { - "priority": 0 - }, - "relaxLocality": true, - "resourceName": "*" - }, - { - "capability": { - "memory": 512, - "virtualCores": 1 - }, - "nodeLabelExpression": "", - "numContainers": 0, - "priority": { - "priority": 0 - }, - "relaxLocality": true, - "resourceName": "/default-rack" - } - ] - }, - { - "id": "application_1459790549146_0002", - "user": "root", - "name": "protodeserializer_1", - "queue": "default", - "state": "KILLED", - "finalStatus": "KILLED", - "progress": 100, - "trackingUI": "History", - "trackingUrl": "http://yarnrm:8088/cluster/app/application_1459790549146_0002", - "diagnostics": "Application killed by user.", - "clusterId": 1459790549146, - "applicationType": "Samza", - "applicationTags": "", - "startedTime": 1459791820396, - "finishedTime": 1459792284264, - "elapsedTime": 463868, - "amContainerLogs": "http://yarnnm:8042/node/containerlogs/container_1459790549146_0002_01_000001/root", - "amHostHttpAddress": "yarnnm:8042", - "allocatedMB": -1, - "allocatedVCores": -1, - "runningContainers": -1, - "memorySeconds": 462177, - "vcoreSeconds": 902, - "preemptedResourceMB": 0, - "preemptedResourceVCores": 0, - "numNonAMContainerPreempted": 0, - "numAMContainerPreempted": 0 - }, - { - "id": "application_1459790549146_0001", - "user": "root", - "name": "protodeserializer_1", - "queue": "default", - "state": "KILLED", - "finalStatus": "KILLED", - "progress": 100, - "trackingUI": "History", - "trackingUrl": "http://yarnrm:8088/cluster/app/application_1459790549146_0001", - "diagnostics": "Application killed by user.", - "clusterId": 1459790549146, - "applicationType": "Samza", - "applicationTags": "", - "startedTime": 1459791108916, - "finishedTime": 1459791813659, - "elapsedTime": 704743, - "amContainerLogs": "http://yarnnm:8042/node/containerlogs/container_1459790549146_0001_01_000001/root", - "amHostHttpAddress": "yarnnm:8042", - "allocatedMB": -1, - "allocatedVCores": -1, - "runningContainers": -1, - "memorySeconds": 711605, - "vcoreSeconds": 1389, - "preemptedResourceMB": 0, - "preemptedResourceVCores": 0, - "numNonAMContainerPreempted": 0, - "numAMContainerPreempted": 0 - } - ] - } -} \ No newline at end of file diff --git a/samza-shell/src/main/bash/run-config-manager.sh b/samza-shell/src/main/bash/run-config-manager.sh deleted file mode 100755 index 96777e7105..0000000000 --- a/samza-shell/src/main/bash/run-config-manager.sh +++ /dev/null @@ -1,25 +0,0 @@ -#!/bin/bash -# 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. - -if [[ -n $(find "$base_dir/lib" -regex ".*samza-log4j2.*.jar*") ]]; then - [[ $JAVA_OPTS != *-Dlog4j.configurationFile* ]] && export JAVA_OPTS="$JAVA_OPTS -Dlog4j.configurationFile=file:$(dirname $0)/log4j2-console.xml" -elif [[ -n $(find "$base_dir/lib" -regex ".*samza-log4j.*.jar*") ]]; then - [[ $JAVA_OPTS != *-Dlog4j.configuration* ]] && export JAVA_OPTS="$JAVA_OPTS -Dlog4j.configuration=file:$(dirname $0)/log4j-console.xml" -fi - -exec $(dirname $0)/run-class.sh org.apache.samza.autoscaling.deployer.ConfigManager "$@" diff --git a/settings.gradle b/settings.gradle index c636706d71..cf4c9be7a3 100644 --- a/settings.gradle +++ b/settings.gradle @@ -23,7 +23,6 @@ include \ 'samza-shell' def scalaModules = [ - 'samza-autoscaling', 'samza-aws', 'samza-azure', 'samza-core', diff --git a/sonar-project.properties b/sonar-project.properties index 6e420dad57..2e9c7bed16 100644 --- a/sonar-project.properties +++ b/sonar-project.properties @@ -32,4 +32,4 @@ sonar.tests=src/test sonar.jacoco.reportPaths=build/jacoco/test.exec # List of subprojects here -sonar.modules=samza-api,samza-autoscaling,samza-azure,samza-core,samza-elasticsearch,samza-hdfs,samza-kafka,samza-kv-inmemory,samza-kv-rocksdb,samza-kv-couchbase,samza-kv,samza-log4j,samza-rest,samza-shell,samza-test,samza-yarn +sonar.modules=samza-api,samza-azure,samza-core,samza-elasticsearch,samza-hdfs,samza-kafka,samza-kv-inmemory,samza-kv-rocksdb,samza-kv-couchbase,samza-kv,samza-log4j,samza-rest,samza-shell,samza-test,samza-yarn From afcb11c85bd8f0e0839dc846fc94191fc0323407 Mon Sep 17 00:00:00 2001 From: Shanthoosh Venkataraman Date: Tue, 14 Jan 2020 16:20:29 -0800 Subject: [PATCH 39/49] Fix the coordinator stream creation workflow. --- .../samza/system/kafka/KafkaSystemAdmin.java | 10 ++--- .../kafka/TestKafkaSystemAdminJava.java | 39 +++++++++++++++++++ 2 files changed, 44 insertions(+), 5 deletions(-) diff --git a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaSystemAdmin.java b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaSystemAdmin.java index 97229db9eb..e5d6af18ff 100644 --- a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaSystemAdmin.java +++ b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaSystemAdmin.java @@ -464,19 +464,19 @@ public boolean createStream(StreamSpec streamSpec) { LOG.info("Creating Kafka topic: {} on system: {}", streamSpec.getPhysicalName(), streamSpec.getSystemName()); final String REPL_FACTOR = "replication.factor"; - KafkaStreamSpec kSpec = toKafkaSpec(streamSpec); - String topicName = kSpec.getPhysicalName(); + KafkaStreamSpec kafkaStreamSpec = toKafkaSpec(streamSpec); + String topicName = kafkaStreamSpec.getPhysicalName(); // create topic. - NewTopic newTopic = new NewTopic(topicName, kSpec.getPartitionCount(), (short) kSpec.getReplicationFactor()); + NewTopic newTopic = new NewTopic(topicName, kafkaStreamSpec.getPartitionCount(), (short) kafkaStreamSpec.getReplicationFactor()); // specify the configs - Map streamConfig = new HashMap<>(streamSpec.getConfig()); + Map streamConfig = new HashMap<>(kafkaStreamSpec.getConfig()); // HACK - replication.factor is invalid config for AdminClient.createTopics if (streamConfig.containsKey(REPL_FACTOR)) { String repl = streamConfig.get(REPL_FACTOR); LOG.warn("Configuration {}={} for topic={} is invalid. Using kSpec repl factor {}", - REPL_FACTOR, repl, kSpec.getPhysicalName(), kSpec.getReplicationFactor()); + REPL_FACTOR, repl, kafkaStreamSpec.getPhysicalName(), kafkaStreamSpec.getReplicationFactor()); streamConfig.remove(REPL_FACTOR); } newTopic.configs(new MapConfig(streamConfig)); diff --git a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java index 143160000e..7ca03f3ae2 100644 --- a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java +++ b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java @@ -19,15 +19,22 @@ package org.apache.samza.system.kafka; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import java.util.Collection; import java.util.HashMap; +import java.util.List; import java.util.Map; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.kafka.clients.admin.DescribeConfigsResult; import org.apache.kafka.clients.admin.DescribeTopicsResult; import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.config.TopicConfig; import org.apache.samza.Partition; import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; @@ -61,6 +68,38 @@ public class TestKafkaSystemAdminJava extends TestKafkaSystemAdmin { private static final SystemStreamPartition TEST_SYSTEM_STREAM_PARTITION = new SystemStreamPartition(TEST_SYSTEM, TEST_STREAM, TEST_PARTITION); private static final String TEST_OFFSET = "10"; + @Test + public void testCreateStreamShouldCoordinatorStreamWithCorrectTopicProperties() throws Exception { + String coordinatorTopicName = String.format("topic-name-%s", RandomStringUtils.randomAlphabetic(5)); + StreamSpec coordinatorStreamSpec = KafkaStreamSpec.createCoordinatorStreamSpec(coordinatorTopicName, SYSTEM()); + + boolean hasCreatedStream = systemAdmin().createStream(coordinatorStreamSpec); + + assertTrue(hasCreatedStream); + + Map coordinatorTopicProperties = getTopicConfigFromKafkaBroker(coordinatorTopicName); + + assertEquals("compact", coordinatorTopicProperties.get(TopicConfig.CLEANUP_POLICY_CONFIG)); + assertEquals("26214400", coordinatorTopicProperties.get(TopicConfig.SEGMENT_BYTES_CONFIG)); + assertEquals("86400000", coordinatorTopicProperties.get(TopicConfig.DELETE_RETENTION_MS_CONFIG)); + } + + private static Map getTopicConfigFromKafkaBroker(String topicName) throws Exception { + List configResourceList = ImmutableList.of( + new ConfigResource(ConfigResource.Type.TOPIC, topicName)); + Map configResourceConfigMap = + adminClient().describeConfigs(configResourceList).all().get(); + Map kafkaTopicConfig = new HashMap<>(); + + configResourceConfigMap.values().forEach(configEntry -> { + configEntry.entries().forEach(config -> { + kafkaTopicConfig.put(config.name(), config.value()); + }); + }); + + return kafkaTopicConfig; + } + @Test public void testGetOffsetsAfter() { SystemStreamPartition ssp1 = new SystemStreamPartition(SYSTEM, TOPIC, new Partition(0)); From a35de7154460b405d1afe57046f4562f7f55e121 Mon Sep 17 00:00:00 2001 From: shanthoosh Date: Tue, 14 Jan 2020 19:02:13 -0800 Subject: [PATCH 40/49] SAMZA-2431: Fix the checkpoint and changelog topic auto-creation. (#1251) * Fix the checkpoint and changelog topic creation configurations. * Address review comments. * Address review comments. --- .../org/apache/samza/system/StreamSpec.java | 4 +- .../samza/system/kafka/KafkaSystemAdmin.java | 7 ++- .../org/apache/samza/config/KafkaConfig.scala | 6 ++- .../kafka/TestKafkaSystemAdminJava.java | 26 +++++++++++ .../apache/samza/config/TestKafkaConfig.scala | 46 ++++++++++++++++++- 5 files changed, 82 insertions(+), 7 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java index a1ad5e4cde..c122371484 100644 --- a/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java +++ b/samza-api/src/main/java/org/apache/samza/system/StreamSpec.java @@ -19,6 +19,8 @@ package org.apache.samza.system; +import com.google.common.base.Joiner; + import java.io.Serializable; import java.util.Collections; import java.util.HashMap; @@ -269,6 +271,6 @@ public static StreamSpec createStreamAppenderStreamSpec(String physicalName, Str @Override public String toString() { - return String.format("StreamSpec: id=%s, systemName=%s, pName=%s, partCount=%d.", id, systemName, physicalName, partitionCount); + return String.format("StreamSpec: id=%s, systemName=%s, pName=%s, partCount=%d, config=%s.", id, systemName, physicalName, partitionCount, Joiner.on(",").withKeyValueSeparator("=").join(config)); } } diff --git a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaSystemAdmin.java b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaSystemAdmin.java index e5d6af18ff..ecb95a92fb 100644 --- a/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaSystemAdmin.java +++ b/samza-kafka/src/main/java/org/apache/samza/system/kafka/KafkaSystemAdmin.java @@ -541,8 +541,11 @@ public KafkaStreamSpec toKafkaSpec(StreamSpec spec) { new KafkaStreamSpec(spec.getId(), spec.getPhysicalName(), systemName, 1, coordinatorStreamReplicationFactor, coordinatorStreamProperties); } else if (spec.isCheckpointStream()) { - kafkaSpec = KafkaStreamSpec.fromSpec(StreamSpec.createCheckpointStreamSpec(spec.getPhysicalName(), systemName)) - .copyWithReplicationFactor(Integer.parseInt(new KafkaConfig(config).getCheckpointReplicationFactor().get())); + Properties checkpointTopicProperties = new Properties(); + checkpointTopicProperties.putAll(spec.getConfig()); + kafkaSpec = KafkaStreamSpec.fromSpec(StreamSpec.createCheckpointStreamSpec(spec.getPhysicalName(), spec.getSystemName())) + .copyWithReplicationFactor(Integer.parseInt(new KafkaConfig(config).getCheckpointReplicationFactor().get())) + .copyWithProperties(checkpointTopicProperties); } else if (intermediateStreamProperties.containsKey(spec.getId())) { kafkaSpec = KafkaStreamSpec.fromSpec(spec); Properties properties = kafkaSpec.getProperties(); diff --git a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala index 75fbb6b27f..3b5f5f3d89 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala @@ -316,7 +316,6 @@ class KafkaConfig(config: Config) extends ScalaMapConfig(config) { val filteredConfigs = config.subset(KafkaConfig.CHANGELOG_STREAM_KAFKA_SETTINGS format name, true) val kafkaChangeLogProperties = new Properties - val appConfig = new ApplicationConfig(config) // SAMZA-1600: do not use the combination of "compact,delete" as cleanup policy until we pick up Kafka broker 0.11.0.57, // 1.0.2, or 1.1.0 (see KAFKA-6568) @@ -325,7 +324,10 @@ class KafkaConfig(config: Config) extends ScalaMapConfig(config) { // - Set topic TTL to be the same as RocksDB TTL Option(config.get("stores.%s.rocksdb.ttl.ms" format name)) match { case Some(rocksDbTtl) => - if (!config.containsKey("stores.%s.changelog.kafka.cleanup.policy" format name)) { + if (!rocksDbTtl.isEmpty && rocksDbTtl.toInt < 0) { + kafkaChangeLogProperties.setProperty("cleanup.policy", "compact") + kafkaChangeLogProperties.setProperty("max.message.bytes", getChangelogStreamMaxMessageByte(name)) + } else if (!config.containsKey("stores.%s.changelog.kafka.cleanup.policy" format name)) { kafkaChangeLogProperties.setProperty("cleanup.policy", "delete") if (!config.containsKey("stores.%s.changelog.kafka.retention.ms" format name)) { kafkaChangeLogProperties.setProperty("retention.ms", String.valueOf(rocksDbTtl)) diff --git a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java index 7ca03f3ae2..82d635f05e 100644 --- a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java +++ b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java @@ -114,6 +114,32 @@ public void testGetOffsetsAfter() { Assert.assertEquals("3", offsets.get(ssp2)); } + @Test + public void testToKafkaSpecForCheckpointStreamShouldReturnTheCorrectStreamSpecByPreservingTheConfig() { + String topicName = "testStream"; + String streamId = "samza-internal-checkpoint-stream-id"; + int partitionCount = 1; + Map map = new HashMap<>(); + map.put("cleanup.policy", "compact"); + map.put("replication.factor", "3"); + map.put("segment.bytes", "536870912"); + map.put("delete.retention.ms", "86400000"); + + Config config = new MapConfig(map); + + StreamSpec spec = new StreamSpec(streamId, topicName, SYSTEM, partitionCount, config); + KafkaSystemAdmin kafkaSystemAdmin = systemAdmin(); + KafkaStreamSpec kafkaStreamSpec = kafkaSystemAdmin.toKafkaSpec(spec); + System.out.println(kafkaStreamSpec); + assertEquals(streamId, kafkaStreamSpec.getId()); + assertEquals(topicName, kafkaStreamSpec.getPhysicalName()); + assertEquals(partitionCount, kafkaStreamSpec.getPartitionCount()); + assertEquals(3, kafkaStreamSpec.getReplicationFactor()); + assertEquals("compact", kafkaStreamSpec.getConfig().get("cleanup.policy")); + assertEquals("536870912", kafkaStreamSpec.getConfig().get("segment.bytes")); + assertEquals("86400000", kafkaStreamSpec.getConfig().get("delete.retention.ms")); + } + @Test public void testToKafkaSpec() { String topicName = "testStream"; diff --git a/samza-kafka/src/test/scala/org/apache/samza/config/TestKafkaConfig.scala b/samza-kafka/src/test/scala/org/apache/samza/config/TestKafkaConfig.scala index 8558a85a2f..00b103d817 100644 --- a/samza-kafka/src/test/scala/org/apache/samza/config/TestKafkaConfig.scala +++ b/samza-kafka/src/test/scala/org/apache/samza/config/TestKafkaConfig.scala @@ -22,14 +22,14 @@ package org.apache.samza.config import java.util.Properties import java.util.concurrent.TimeUnit -import org.apache.samza.config.factories.PropertiesConfigFactory import org.junit.Assert._ +import org.junit.After +import org.junit.Before import org.junit.Test import scala.collection.JavaConverters._ import org.apache.kafka.common.serialization.ByteArraySerializer import org.apache.kafka.clients.producer.ProducerConfig -import org.junit.Before class TestKafkaConfig { @@ -47,6 +47,10 @@ class TestKafkaConfig { props.setProperty(JobConfig.JOB_NAME, "jobName") } + @After + def clearUpProperties(): Unit = { + props.clear() + } @Test def testStreamLevelFetchSizeOverride() { @@ -81,6 +85,44 @@ class TestKafkaConfig { assertEquals("65536", kafkaConfig4.getConsumerFetchThresholdBytes("kafka").get) } + @Test + def testChangeLogPropertiesShouldReturnCorrectTopicConfigurationForInfiniteTTLStores(): Unit = { + val props = new Properties + props.setProperty(KAFKA_PRODUCER_PROPERTY_PREFIX + "bootstrap.servers", "localhost:9092") + props.setProperty("systems." + SYSTEM_NAME + ".consumer.zookeeper.connect", "localhost:2181/") + props.setProperty(JobConfig.JOB_NAME, "jobName") + + props.setProperty("stores.test1.changelog", "kafka.mychangelog1") + props.setProperty("stores.test1.rocksdb.ttl.ms", "-1") + + val mapConfig = new MapConfig(props.asScala.asJava) + val kafkaConfig = new KafkaConfig(mapConfig) + val kafkaProperties = kafkaConfig.getChangelogKafkaProperties("test1") + assertEquals("compact", kafkaProperties.getProperty("cleanup.policy")) + assertEquals("536870912", kafkaProperties.getProperty("segment.bytes")) + assertEquals("1000012", kafkaProperties.getProperty("max.message.bytes")) + assertEquals("86400000", kafkaProperties.getProperty("delete.retention.ms")) + } + + @Test + def testChangeLogPropertiesShouldReturnCorrectTopicConfigurationForStoresWithEmptyRocksDBTTL(): Unit = { + val props = new Properties + props.setProperty(KAFKA_PRODUCER_PROPERTY_PREFIX + "bootstrap.servers", "localhost:9092") + props.setProperty("systems." + SYSTEM_NAME + ".consumer.zookeeper.connect", "localhost:2181/") + props.setProperty(JobConfig.JOB_NAME, "jobName") + + props.setProperty("stores.test1.changelog", "kafka.mychangelog1") + + val mapConfig = new MapConfig(props.asScala.asJava) + val kafkaConfig = new KafkaConfig(mapConfig) + val kafkaProperties = kafkaConfig.getChangelogKafkaProperties("test1") + assertEquals("compact", kafkaProperties.getProperty("cleanup.policy")) + assertEquals("536870912", kafkaProperties.getProperty("segment.bytes")) + assertEquals("86400000", kafkaProperties.getProperty("delete.retention.ms")) + assertEquals("1000012", kafkaProperties.getProperty("max.message.bytes")) + + } + @Test def testChangeLogProperties() { props.setProperty("job.changelog.system", SYSTEM_NAME) From 405d8a648cbc103b5e28974d7fd9a326d65d1214 Mon Sep 17 00:00:00 2001 From: Abhishek Shivanna Date: Wed, 15 Jan 2020 14:14:53 -0800 Subject: [PATCH 41/49] Disable Flaky Tests --- .../org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java b/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java index 26ba0eca98..d46d3b16a2 100644 --- a/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java +++ b/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java @@ -102,6 +102,7 @@ public void testEndToEndWithSystemMessages() throws SamzaSqlValidatorException { Assert.assertEquals(numMessages, outMessages.size()); } + @Ignore @Test public void testEndToEndDisableSystemMessages() throws SamzaSqlValidatorException { int numMessages = 20; @@ -258,6 +259,7 @@ public void testEndToEndFanIn() throws SamzaSqlValidatorException { Assert.assertTrue(IntStream.range(0, numMessages).boxed().collect(Collectors.toList()).equals(new ArrayList<>(outMessagesSet))); } + @Ignore @Test public void testEndToEndFanOut() throws SamzaSqlValidatorException { int numMessages = 20; @@ -760,6 +762,7 @@ public void testEndToEndStreamTableInnerJoinWithPrimaryKey() throws Exception { Assert.assertEquals(expectedOutMessages, outMessages); } + @Ignore @Test public void testEndToEndStreamTableJoinWithSubQuery() throws Exception { int numMessages = 20; From c867cd38a5f92ed4a575d0021f92d67dd140a1dc Mon Sep 17 00:00:00 2001 From: lakshmi-manasa-g Date: Thu, 16 Jan 2020 10:46:07 -0800 Subject: [PATCH 42/49] SAMZA-2421: Add SystemProducer for Azure Blob Storage (#1239) --- build.gradle | 2 + gradle/dependency-versions.gradle | 2 +- .../azureblob/AzureBlobBasicMetrics.java | 98 +++ .../system/azureblob/AzureBlobConfig.java | 188 ++++++ .../azureblob/AzureBlobSystemAdmin.java | 44 ++ .../azureblob/AzureBlobSystemFactory.java | 56 ++ .../azureblob/avro/AzureBlobAvroWriter.java | 358 +++++++++++ .../avro/AzureBlobAvroWriterFactory.java | 43 ++ .../azureblob/avro/AzureBlobOutputStream.java | 305 +++++++++ .../azureblob/compression/Compression.java | 41 ++ .../compression/CompressionFactory.java | 43 ++ .../compression/CompressionType.java | 31 + .../compression/GzipCompression.java | 75 +++ .../compression/NoneCompression.java | 37 ++ .../producer/AzureBlobSystemProducer.java | 518 +++++++++++++++ .../AzureBlobSystemProducerMetrics.java | 134 ++++ .../azureblob/producer/AzureBlobWriter.java | 52 ++ .../producer/AzureBlobWriterFactory.java | 44 ++ .../producer/AzureBlobWriterMetrics.java | 81 +++ .../avro/TestAzureBlobAvroWriter.java | 528 ++++++++++++++++ .../avro/TestAzureBlobOutputStream.java | 346 ++++++++++ .../compression/TestGzipCompression.java | 76 +++ .../producer/TestAzureBlobSystemProducer.java | 594 ++++++++++++++++++ 23 files changed, 3695 insertions(+), 1 deletion(-) create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobBasicMetrics.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobConfig.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemAdmin.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemFactory.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriterFactory.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/Compression.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionFactory.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionType.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/GzipCompression.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/NoneCompression.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducer.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducerMetrics.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriter.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriterFactory.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriterMetrics.java create mode 100644 samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java create mode 100644 samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java create mode 100644 samza-azure/src/test/java/org/apache/samza/system/azureblob/compression/TestGzipCompression.java create mode 100644 samza-azure/src/test/java/org/apache/samza/system/azureblob/producer/TestAzureBlobSystemProducer.java diff --git a/build.gradle b/build.gradle index 3820c8d33f..143a04cab6 100644 --- a/build.gradle +++ b/build.gradle @@ -213,10 +213,12 @@ project(":samza-azure_$scalaSuffix") { apply plugin: 'checkstyle' dependencies { + compile "com.azure:azure-storage-blob:12.0.1" compile "com.microsoft.azure:azure-storage:5.3.1" compile "com.microsoft.azure:azure-eventhubs:1.0.1" compile "com.fasterxml.jackson.core:jackson-core:2.8.8" compile "io.dropwizard.metrics:metrics-core:3.1.2" + compile "org.apache.avro:avro:$avroVersion" compile project(':samza-api') compile project(":samza-core_$scalaSuffix") compile "org.slf4j:slf4j-api:$slf4jVersion" diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index fcf472e97d..c4392b4687 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -18,7 +18,7 @@ */ ext { apacheCommonsCollections4Version = "4.0" - avroVersion = "1.7.1" + avroVersion = "1.7.7" calciteVersion = "1.19.0" commonsCliVersion = "1.2" commonsCodecVersion = "1.9" diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobBasicMetrics.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobBasicMetrics.java new file mode 100644 index 0000000000..388135ee04 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobBasicMetrics.java @@ -0,0 +1,98 @@ +/* + * 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.system.azureblob; + +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.MetricsRegistry; + +/** + * This class holds all the metrics to be measured (like write, write byte, error) + * for a single group (like aggregate, system or source). + */ +public class AzureBlobBasicMetrics { + public static final String EVENT_WRITE_RATE = "eventWriteRate"; + public static final String EVENT_PRODUCE_ERROR = "eventProduceError"; + public static final String EVENT_WRITE_BYTE_RATE = "eventWriteByteRate"; + public static final String EVENT_COMPRESS_BYTE_RATE = "eventCompressByteRate"; + public static final String AZURE_BLOCK_UPLOAD_RATE = "azureBlockUploadRate"; + public static final String AZURE_BLOB_COMMIT_RATE = "azureBlobCommitRate"; + + private final Counter writeMetrics; + private final Counter writeByteMetrics; + private final Counter errorMetrics; + private final Counter compressByteMetrics; + private final Counter azureUploadMetrics; + private final Counter azureCommitMetrics; + + public AzureBlobBasicMetrics(String group, MetricsRegistry metricsRegistry) { + writeMetrics = metricsRegistry.newCounter(group, EVENT_WRITE_RATE); + errorMetrics = metricsRegistry.newCounter(group, EVENT_PRODUCE_ERROR); + writeByteMetrics = metricsRegistry.newCounter(group, EVENT_WRITE_BYTE_RATE); + compressByteMetrics = metricsRegistry.newCounter(group, EVENT_COMPRESS_BYTE_RATE); + azureUploadMetrics = metricsRegistry.newCounter(group, AZURE_BLOCK_UPLOAD_RATE); + azureCommitMetrics = metricsRegistry.newCounter(group, AZURE_BLOB_COMMIT_RATE); + } + + /** + * Increments the write metrics counter by 1. + */ + public void updateWriteMetrics() { + writeMetrics.inc(); + } + + /** + * Increments the write byte metrics counter by the number of bytes written. + * @param dataLength number of bytes written. + */ + public void updateWriteByteMetrics(long dataLength) { + writeByteMetrics.inc(dataLength); + } + + /** + * Increments the compress byte metrics counter by the number of compressed bytes written. + * @param dataLength number of bytes written. + */ + public void updateCompressByteMetrics(long dataLength) { + compressByteMetrics.inc(dataLength); + } + + /** + * Increments the error metrics counter by 1. + */ + public void updateErrorMetrics() { + errorMetrics.inc(); + } + + + /** + * Increments the azure block upload metrics counter by 1. + */ + public void updateAzureUploadMetrics() { + azureUploadMetrics.inc(); + } + + + /** + * Increments the azure blob commit metrics counter by 1. + */ + public void updateAzureCommitMetrics() { + azureCommitMetrics.inc(); + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobConfig.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobConfig.java new file mode 100644 index 0000000000..7c468c6e24 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobConfig.java @@ -0,0 +1,188 @@ +/* + * 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.system.azureblob; + +import org.apache.samza.system.azureblob.compression.CompressionType; +import java.time.Duration; +import org.apache.samza.config.Config; +import org.apache.samza.config.ConfigException; +import org.apache.samza.config.MapConfig; + +public class AzureBlobConfig extends MapConfig { + private static final String SYSTEM_AZUREBLOB_PREFIX = "systems.%s.azureblob."; + //Server Instance Level Property + + // The duration after which an Azure request will be logged as a warning. + public static final String AZURE_BLOB_LOG_SLOW_REQUESTS_MS = "samza.azureblob.log.slowRequestMs"; + private static final long AZURE_BLOB_LOG_SLOW_REQUESTS_MS_DEFAULT = Duration.ofSeconds(30).toMillis(); + + // system Level Properties. + // fully qualified class name of the AzureBlobWriter impl for the producer system + public static final String SYSTEM_WRITER_FACTORY_CLASS_NAME = SYSTEM_AZUREBLOB_PREFIX + "writer.factory.class"; + public static final String SYSTEM_WRITER_FACTORY_CLASS_NAME_DEFAULT = "org.apache.samza.system.azureblob.avro.AzureBlobAvroWriterFactory"; + + // Azure Storage Account name under which the Azure container representing this system is. + // System name = Azure container name (https://docs.microsoft.com/en-us/rest/api/storageservices/naming-and-referencing-containers--blobs--and-metadata#container-names) + public static final String SYSTEM_AZURE_ACCOUNT_NAME = SYSTEM_AZUREBLOB_PREFIX + "account.name"; + + // Azure Storage Account key associated with the Azure Storage Account + public static final String SYSTEM_AZURE_ACCOUNT_KEY = SYSTEM_AZUREBLOB_PREFIX + "account.key"; + + // Whether to use proxy while connecting to Azure Storage + public static final String SYSTEM_AZURE_USE_PROXY = SYSTEM_AZUREBLOB_PREFIX + "proxy.use"; + public static final boolean SYSTEM_AZURE_USE_PROXY_DEFAULT = false; + + // name of the host to be used as proxy + public static final String SYSTEM_AZURE_PROXY_HOSTNAME = SYSTEM_AZUREBLOB_PREFIX + "proxy.hostname"; + + // port in the proxy host to be used + public static final String SYSTEM_AZURE_PROXY_PORT = SYSTEM_AZUREBLOB_PREFIX + "proxy.port"; + + // type of compression to be used before uploading blocks : “none” or “gzip” + public static final String SYSTEM_COMPRESSION_TYPE = SYSTEM_AZUREBLOB_PREFIX + "compression.type"; + private static final CompressionType SYSTEM_COMPRESSION_TYPE_DEFAULT = CompressionType.NONE; + + // maximum size of uncompressed block in bytes + public static final String SYSTEM_MAX_FLUSH_THRESHOLD_SIZE = SYSTEM_AZUREBLOB_PREFIX + "maxFlushThresholdSize"; + private static final int SYSTEM_MAX_FLUSH_THRESHOLD_SIZE_DEFAULT = 10485760; + + // maximum size of uncompressed blob in bytes + public static final String SYSTEM_MAX_BLOB_SIZE = SYSTEM_AZUREBLOB_PREFIX + "maxBlobSize"; + private static final long SYSTEM_MAX_BLOB_SIZE_DEFAULT = Long.MAX_VALUE; // unlimited + + // maximum number of messages in a blob + public static final String SYSTEM_MAX_MESSAGES_PER_BLOB = SYSTEM_AZUREBLOB_PREFIX + "maxMessagesPerBlob"; + private static final long SYSTEM_MAX_MESSAGES_PER_BLOB_DEFAULT = Long.MAX_VALUE; // unlimited + + // number of threads to asynchronously upload blocks + public static final String SYSTEM_THREAD_POOL_COUNT = SYSTEM_AZUREBLOB_PREFIX + "threadPoolCount"; + private static final int SYSTEM_THREAD_POOL_COUNT_DEFAULT = 1; + + // size of the queue to hold blocks ready to be uploaded by asynchronous threads. + // If all threads are busy uploading then blocks are queued and if queue is full then main thread will start uploading + // which will block processing of incoming messages + // Default - Thread Pool Count * 2 + public static final String SYSTEM_BLOCKING_QUEUE_SIZE = SYSTEM_AZUREBLOB_PREFIX + "blockingQueueSize"; + + // timeout to finish uploading all blocks before committing a blob + public static final String SYSTEM_FLUSH_TIMEOUT_MS = SYSTEM_AZUREBLOB_PREFIX + "flushTimeoutMs"; + private static final long SYSTEM_FLUSH_TIMEOUT_MS_DEFAULT = Duration.ofMinutes(3).toMillis(); + + // timeout to finish committing all the blobs currently being written to. This does not include the flush timeout per blob + public static final String SYSTEM_CLOSE_TIMEOUT_MS = SYSTEM_AZUREBLOB_PREFIX + "closeTimeoutMs"; + private static final long SYSTEM_CLOSE_TIMEOUT_MS_DEFAULT = Duration.ofMinutes(5).toMillis(); + + // if true, a random string of 8 chars is suffixed to the blob name to prevent name collision + // when more than one Samza tasks are writing to the same SSP. + public static final String SYSTEM_SUFFIX_RANDOM_STRING_TO_BLOB_NAME = SYSTEM_AZUREBLOB_PREFIX + "suffixRandomStringToBlobName"; + private static final boolean SYSTEM_SUFFIX_RANDOM_STRING_TO_BLOB_NAME_DEFAULT = true; + + public AzureBlobConfig(Config config) { + super(config); + } + + public String getAzureAccountKey(String systemName) { + String accountKey = get(String.format(SYSTEM_AZURE_ACCOUNT_KEY, systemName)); + if (accountKey == null) { + throw new ConfigException("Azure account key is required."); + } + return accountKey; + } + + public String getAzureAccountName(String systemName) { + String accountName = get(String.format(SYSTEM_AZURE_ACCOUNT_NAME, systemName)); + if (accountName == null) { + throw new ConfigException("Azure account name is required."); + } + return accountName; + } + + public boolean getUseProxy(String systemName) { + return getBoolean(String.format(SYSTEM_AZURE_USE_PROXY, systemName), SYSTEM_AZURE_USE_PROXY_DEFAULT); + } + + public String getAzureProxyHostname(String systemName) { + String hostname = get(String.format(SYSTEM_AZURE_PROXY_HOSTNAME, systemName)); + if (hostname == null) { + throw new ConfigException("Azure proxy host name is required."); + } + return hostname; + } + + public int getAzureProxyPort(String systemName) { + return getInt(String.format(SYSTEM_AZURE_PROXY_PORT, systemName)); + } + + public CompressionType getCompressionType(String systemName) { + return CompressionType.valueOf(get(String.format(SYSTEM_COMPRESSION_TYPE, systemName), SYSTEM_COMPRESSION_TYPE_DEFAULT.name()).toUpperCase()); + } + + public String getAzureBlobWriterFactoryClassName(String systemName) { + return get(String.format(SYSTEM_WRITER_FACTORY_CLASS_NAME, systemName), SYSTEM_WRITER_FACTORY_CLASS_NAME_DEFAULT); + } + + public int getMaxFlushThresholdSize(String systemName) { + return getInt(String.format(SYSTEM_MAX_FLUSH_THRESHOLD_SIZE, systemName), SYSTEM_MAX_FLUSH_THRESHOLD_SIZE_DEFAULT); + } + + public int getAzureBlobThreadPoolCount(String systemName) { + return getInt(String.format(SYSTEM_THREAD_POOL_COUNT, systemName), SYSTEM_THREAD_POOL_COUNT_DEFAULT); + } + + public int getBlockingQueueSize(String systemName) { + return getInt(String.format(SYSTEM_BLOCKING_QUEUE_SIZE, systemName), 2 * getAzureBlobThreadPoolCount(systemName)); + } + + public long getFlushTimeoutMs(String systemName) { + long timeout = getLong(String.format(SYSTEM_FLUSH_TIMEOUT_MS, systemName), SYSTEM_FLUSH_TIMEOUT_MS_DEFAULT); + if (timeout <= 0) { + throw new ConfigException("Azure Blob flush timeout can not be <= 0"); + } + return timeout; + } + + public long getCloseTimeoutMs(String systemName) { + long timeout = getLong(String.format(SYSTEM_CLOSE_TIMEOUT_MS, systemName), SYSTEM_CLOSE_TIMEOUT_MS_DEFAULT); + if (timeout <= 0) { + throw new ConfigException("Azure Blob close timeout can not be <= 0"); + } + return timeout; + } + + public long getLogSlowRequestsMs() { + long duration = getLong(AZURE_BLOB_LOG_SLOW_REQUESTS_MS, AZURE_BLOB_LOG_SLOW_REQUESTS_MS_DEFAULT); + if (duration <= 0) { + throw new ConfigException("Azure blob duration to log slow requests can not be <=0."); + } + return duration; + } + + public boolean getSuffixRandomStringToBlobName(String systemName) { + return getBoolean(String.format(SYSTEM_SUFFIX_RANDOM_STRING_TO_BLOB_NAME, systemName), SYSTEM_SUFFIX_RANDOM_STRING_TO_BLOB_NAME_DEFAULT); + } + + public long getMaxBlobSize(String systemName) { + return getLong(String.format(SYSTEM_MAX_BLOB_SIZE, systemName), SYSTEM_MAX_BLOB_SIZE_DEFAULT); + } + + public long getMaxMessagesPerBlob(String systemName) { + return getLong(String.format(SYSTEM_MAX_MESSAGES_PER_BLOB, systemName), SYSTEM_MAX_MESSAGES_PER_BLOB_DEFAULT); + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemAdmin.java new file mode 100644 index 0000000000..ee3f5a223c --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemAdmin.java @@ -0,0 +1,44 @@ +/* + * 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.system.azureblob; + +import java.util.Map; +import java.util.Set; +import org.apache.samza.system.SystemAdmin; +import org.apache.samza.system.SystemStreamMetadata; +import org.apache.samza.system.SystemStreamPartition; + + +/** + * {@inheritDoc} + */ +public class AzureBlobSystemAdmin implements SystemAdmin { + public Map getOffsetsAfter(Map offsets) { + throw new UnsupportedOperationException("getOffsetsAfter not supported for AzureBlobSystemAdmin"); + } + + public Map getSystemStreamMetadata(Set streamNames) { + throw new UnsupportedOperationException("getSystemStreamMetadata not supported for AzureBlobSystemAdmin"); + } + + public Integer offsetComparator(String offset1, String offset2) { + throw new UnsupportedOperationException("offsetComparator not supported for AzureBlobSystemAdmin"); + } +} \ No newline at end of file diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemFactory.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemFactory.java new file mode 100644 index 0000000000..512fe46181 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemFactory.java @@ -0,0 +1,56 @@ +/* + * 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.system.azureblob; + +import org.apache.samza.system.azureblob.producer.AzureBlobSystemProducer; +import org.apache.samza.config.Config; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.system.SystemAdmin; +import org.apache.samza.system.SystemConsumer; +import org.apache.samza.system.SystemFactory; +import org.apache.samza.system.SystemProducer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Do not use this SystemProducer for Coordinator stream store/producer and KafkaCheckpointManager + * as their usage of SystemProducer is a bit inconsistent with this implementation and they also couple + * a SystemProducer with a SystemConsumer which is out of scope for this Factory. + * {@inheritDoc} + */ +public class AzureBlobSystemFactory implements SystemFactory { + private static final Logger LOG = LoggerFactory.getLogger(AzureBlobSystemFactory.class.getName()); + + @Override + public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) { + throw new UnsupportedOperationException("SystemConsumer not supported for AzureBlob!"); + } + + @Override + public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) { + AzureBlobConfig azureBlobConfig = new AzureBlobConfig(config); + return new AzureBlobSystemProducer(systemName, azureBlobConfig, registry); + } + + @Override + public SystemAdmin getAdmin(String systemName, Config config) { + return new AzureBlobSystemAdmin(); + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java new file mode 100644 index 0000000000..7f9a92684a --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java @@ -0,0 +1,358 @@ +/* + * 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.system.azureblob.avro; + +import com.azure.storage.blob.BlobContainerAsyncClient; +import com.azure.storage.blob.specialized.BlockBlobAsyncClient; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.samza.system.azureblob.compression.Compression; +import org.apache.samza.system.azureblob.producer.AzureBlobWriter; +import org.apache.samza.system.azureblob.producer.AzureBlobWriterMetrics; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.TimeZone; +import java.util.UUID; +import java.util.concurrent.Executor; +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.io.EncoderFactory; +import org.apache.avro.specific.SpecificDatumWriter; +import org.apache.avro.specific.SpecificRecord; +import org.apache.samza.SamzaException; +import org.apache.samza.system.OutgoingMessageEnvelope; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class implements {@link org.apache.samza.system.azureblob.producer.AzureBlobWriter} + * for writing avro records to Azure Blob Storage. + * + * It uses {@link org.apache.avro.file.DataFileWriter} to convert avro records it receives to byte[]. + * This byte[] is passed on to {@link org.apache.samza.system.azureblob.avro.AzureBlobOutputStream}. + * AzureBlobOutputStream in turn uploads data to Storage as a blob. + * + * It also accepts encoded records as byte[] as long as the first OutgoingMessageEnvelope this writer receives + * is a decoded record from which to get the schema and record type (GenericRecord vs SpecificRecord). + * The subsequent encoded records are written directly to AzureBlobOutputStream without checking if they conform + * to the schema. It is the responsibility of the user to ensure this. Failing to do so may result in an + * unreadable avro blob. + * + * It expects all OutgoingMessageEnvelopes to be of the same schema. + * To handle schema evolution (sending envelopes of different schema), this writer has to be closed and a new writer + * has to be created. The first envelope of the new writer should contain a valid record to get schema from. + * If used by AzureBlobSystemProducer, this is done through systemProducer.flush(source). + * + * Once closed this object can not be used. + * This is a thread safe class. + * + * If the number of records or size of the current blob exceeds the specified limits then a new blob is created. + */ +public class AzureBlobAvroWriter implements AzureBlobWriter { + private static final Logger LOG = LoggerFactory.getLogger(AzureBlobAvroWriter.class); + private static final String PUBLISHED_FILE_NAME_DATE_FORMAT = "yyyy/MM/dd/HH/mm-ss"; + private static final String BLOB_NAME_AVRO = "%s/%s.avro%s"; + private static final String BLOB_NAME_RANDOM_STRING_AVRO = "%s/%s-%s.avro%s"; + private static final SimpleDateFormat UTC_FORMATTER = buildUTCFormatter(); + + // Avro's DataFileWriter has internal buffers and also adds metadata. + // Based on the current default sizes of these buffers and metadata, the data overhead is a little less than 100KB + // However, taking the overhead to be capped at 1MB to ensure enough room if the default values are increased. + static final long DATAFILEWRITER_OVERHEAD = 1000000; // 1MB + + // currentBlobWriterComponents == null only for the first blob immediately after this AzureBlobAvroWriter object has been created. + // rest of this object's lifecycle, currentBlobWriterComponents is not null. + private BlobWriterComponents currentBlobWriterComponents = null; + private final List allBlobWriterComponents = new ArrayList<>(); + private Schema schema = null; + // datumWriter == null only for the first blob immediately after this AzureBlobAvroWriter object has been created. + // It is created from the schema taken from the first OutgoingMessageEnvelope. Hence the first OME has to be a decoded avro record. + // For rest of this object's lifecycle, datumWriter is not null. + private DatumWriter datumWriter = null; + private volatile boolean isClosed = false; + + private final Executor blobThreadPool; + private final AzureBlobWriterMetrics metrics; + private final int maxBlockFlushThresholdSize; + private final long flushTimeoutMs; + private final Compression compression; + private final BlobContainerAsyncClient containerAsyncClient; + private final String blobURLPrefix; + private final long maxBlobSize; + private final long maxRecordsPerBlob; + private final boolean useRandomStringInBlobName; + private final Object currentDataFileWriterLock = new Object(); + private volatile long recordsInCurrentBlob = 0; + + public AzureBlobAvroWriter(BlobContainerAsyncClient containerAsyncClient, String blobURLPrefix, + Executor blobThreadPool, AzureBlobWriterMetrics metrics, + int maxBlockFlushThresholdSize, long flushTimeoutMs, Compression compression, boolean useRandomStringInBlobName, + long maxBlobSize, long maxRecordsPerBlob) { + + this.blobThreadPool = blobThreadPool; + this.metrics = metrics; + this.maxBlockFlushThresholdSize = maxBlockFlushThresholdSize; + this.flushTimeoutMs = flushTimeoutMs; + this.compression = compression; + this.containerAsyncClient = containerAsyncClient; + this.blobURLPrefix = blobURLPrefix; + this.useRandomStringInBlobName = useRandomStringInBlobName; + this.maxBlobSize = maxBlobSize; + this.maxRecordsPerBlob = maxRecordsPerBlob; + } + + /** + * This method expects the {@link org.apache.samza.system.OutgoingMessageEnvelope} + * to contain a message which is a {@link org.apache.avro.generic.IndexedRecord} or an encoded record aka byte[]. + * If the record is already encoded, it will directly write the byte[] to the output stream without checking if it conforms to schema. + * Else, it encodes the record and writes to output stream. + * However, the first envelope should always be a record and not a byte[]. + * If the blocksize threshold crosses, it will upload the output stream contents as a block. + * If the number of records in current blob or size of current blob exceed limits then a new blob is created. + * Multi-threading and thread-safety: + * The underlying {@link org.apache.avro.file.DataFileWriter} is not thread-safe. + * For this reason, it is essential to wrap accesses to this object in a synchronized block. + * Method write(OutgoingMessageEnvelope) allows multiple threads to encode records as that operation is stateless but + * restricts access to the shared objects through the synchronized block. + * Concurrent access to shared objects is controlled through a common lock and synchronized block and hence ensures + * thread safety. + * @param ome - OutgoingMessageEnvelope that contains the IndexedRecord (GenericRecord or SpecificRecord) or an encoded record as byte[] + * @throws IOException when + * - OutgoingMessageEnvelope's message is not an IndexedRecord or + * - underlying dataFileWriter.append fails + * @throws IllegalStateException when the first OutgoingMessageEnvelope's message is not a record. + */ + @Override + public void write(OutgoingMessageEnvelope ome) throws IOException { + Optional optionalIndexedRecord; + byte[] encodedRecord; + if (ome.getMessage() instanceof IndexedRecord) { + optionalIndexedRecord = Optional.of((IndexedRecord) ome.getMessage()); + encodedRecord = encodeRecord((IndexedRecord) ome.getMessage()); + } else if (ome.getMessage() instanceof byte[]) { + optionalIndexedRecord = Optional.empty(); + encodedRecord = (byte[]) ome.getMessage(); + } else { + throw new IllegalArgumentException("AzureBlobAvroWriter only supports IndexedRecord and byte[]."); + } + + synchronized (currentDataFileWriterLock) { + // if currentBlobWriterComponents is null, then it is the first blob of this AzureBlobAvroWriter object + if (currentBlobWriterComponents == null || willCurrentBlobExceedSize(encodedRecord) || willCurrentBlobExceedRecordLimit()) { + startNextBlob(optionalIndexedRecord); + } + currentBlobWriterComponents.dataFileWriter.appendEncoded(ByteBuffer.wrap(encodedRecord)); + recordsInCurrentBlob++; + } + } + /** + * This method flushes all records written in dataFileWriter to the underlying AzureBlobOutputStream. + * dataFileWriter.flush then explicitly invokes flush of the AzureBlobOutputStream. + * This in turn async uploads content of the output stream as a block and reinits the output stream. + * AzureBlobOutputStream.flush is not ensured if dataFileWriter.flush fails. + * In such a scenario, the current block is not uploaded and blocks uploaded so far are lost. + * {@inheritDoc} + * @throws IOException if underlying dataFileWriter.flush fails + */ + @Override + public void flush() throws IOException { + synchronized (currentDataFileWriterLock) { + currentBlobWriterComponents.dataFileWriter.flush(); + } + } + + /** + * This method closes all DataFileWriters and output streams associated with all the blobs created. + * flush should be explicitly called before close. + * {@inheritDoc} + * @throws IllegalStateException when closing a closed writer + * @throws SamzaException if underlying DataFileWriter.close fails + */ + @Override + public void close() { + synchronized (currentDataFileWriterLock) { + if (isClosed) { + throw new IllegalStateException("Attempting to close an already closed AzureBlobAvroWriter"); + } + allBlobWriterComponents.forEach(blobWriterComponents -> { + try { + closeDataFileWriter(blobWriterComponents.dataFileWriter, blobWriterComponents.azureBlobOutputStream, + blobWriterComponents.blockBlobAsyncClient); + } catch (IOException e) { + throw new SamzaException(e); + } + }); + isClosed = true; + } + } + + @VisibleForTesting + AzureBlobAvroWriter(BlobContainerAsyncClient containerAsyncClient, AzureBlobWriterMetrics metrics, + Executor blobThreadPool, int maxBlockFlushThresholdSize, int flushTimeoutMs, String blobURLPrefix, + DataFileWriter dataFileWriter, + AzureBlobOutputStream azureBlobOutputStream, BlockBlobAsyncClient blockBlobAsyncClient, + long maxBlobSize, long maxRecordsPerBlob, Compression compression, boolean useRandomStringInBlobName) { + if (dataFileWriter == null || azureBlobOutputStream == null || blockBlobAsyncClient == null) { + this.currentBlobWriterComponents = null; + } else { + this.currentBlobWriterComponents = + new BlobWriterComponents(dataFileWriter, azureBlobOutputStream, blockBlobAsyncClient); + } + this.allBlobWriterComponents.add(this.currentBlobWriterComponents); + this.blobThreadPool = blobThreadPool; + this.blobURLPrefix = blobURLPrefix; + this.metrics = metrics; + this.maxBlockFlushThresholdSize = maxBlockFlushThresholdSize; + this.flushTimeoutMs = flushTimeoutMs; + this.compression = compression; + this.containerAsyncClient = containerAsyncClient; + this.useRandomStringInBlobName = useRandomStringInBlobName; + this.maxBlobSize = maxBlobSize; + this.maxRecordsPerBlob = maxRecordsPerBlob; + } + + @VisibleForTesting + byte[] encodeRecord(IndexedRecord record) { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + Schema schema = record.getSchema(); + try { + EncoderFactory encoderfactory = new EncoderFactory(); + BinaryEncoder encoder = encoderfactory.binaryEncoder(out, null); + DatumWriter writer; + if (record instanceof SpecificRecord) { + writer = new SpecificDatumWriter<>(schema); + } else { + writer = new GenericDatumWriter<>(schema); + } + writer.write(record, encoder); + encoder.flush(); //encoder may buffer + } catch (Exception e) { + throw new SamzaException("Unable to serialize Avro record using schema within the record: " + schema.toString(), e); + } + return out.toByteArray(); + } + + private static SimpleDateFormat buildUTCFormatter() { + SimpleDateFormat formatter = new SimpleDateFormat(PUBLISHED_FILE_NAME_DATE_FORMAT); + formatter.setTimeZone(TimeZone.getTimeZone("UTC")); + return formatter; + } + + private void closeDataFileWriter(DataFileWriter dataFileWriter, AzureBlobOutputStream azureBlobOutputStream, + BlockBlobAsyncClient blockBlobAsyncClient) throws IOException { + try { + LOG.info("Closing the blob: {}", blockBlobAsyncClient.getBlobUrl().toString()); + // dataFileWriter.close calls close of the azureBlobOutputStream associated with it. + dataFileWriter.close(); + } catch (Exception e) { + // ensure that close is called even if dataFileWriter.close fails. + // This is to avoid loss of all the blocks uploaded for the blob + // as commitBlockList happens in close of azureBlobOutputStream. + azureBlobOutputStream.close(); + throw e; + } + } + + private void startNextBlob(Optional optionalIndexedRecord) throws IOException { + if (currentBlobWriterComponents != null) { + LOG.info("Starting new blob as current blob size is " + + currentBlobWriterComponents.azureBlobOutputStream.getSize() + + " and max blob size is " + maxBlobSize + + " or number of records is " + recordsInCurrentBlob + + " and max records in blob is " + maxRecordsPerBlob); + currentBlobWriterComponents.dataFileWriter.flush(); + currentBlobWriterComponents.azureBlobOutputStream.releaseBuffer(); + recordsInCurrentBlob = 0; + } + // datumWriter is null when AzureBlobAvroWriter is created but has not yet received a message. + // optionalIndexedRecord is the first message in this case. + if (datumWriter == null) { + if (optionalIndexedRecord.isPresent()) { + IndexedRecord record = optionalIndexedRecord.get(); + schema = record.getSchema(); + if (record instanceof SpecificRecord) { + datumWriter = new SpecificDatumWriter<>(schema); + } else { + datumWriter = new GenericDatumWriter<>(schema); + } + } else { + throw new IllegalStateException("Writing without schema setup."); + } + } + String blobURL; + if (useRandomStringInBlobName) { + blobURL = String.format(BLOB_NAME_RANDOM_STRING_AVRO, blobURLPrefix, + UTC_FORMATTER.format(System.currentTimeMillis()), UUID.randomUUID().toString().substring(0, 8), + compression.getFileExtension()); + } else { + blobURL = String.format(BLOB_NAME_AVRO, blobURLPrefix, + UTC_FORMATTER.format(System.currentTimeMillis()), compression.getFileExtension()); + } + LOG.info("Creating new blob: {}", blobURL); + BlockBlobAsyncClient blockBlobAsyncClient = containerAsyncClient.getBlobAsyncClient(blobURL).getBlockBlobAsyncClient(); + + DataFileWriter dataFileWriter = new DataFileWriter<>(datumWriter); + AzureBlobOutputStream azureBlobOutputStream = new AzureBlobOutputStream(blockBlobAsyncClient, blobThreadPool, metrics, + flushTimeoutMs, maxBlockFlushThresholdSize, compression); + dataFileWriter.create(schema, azureBlobOutputStream); + dataFileWriter.setFlushOnEveryBlock(false); + this.currentBlobWriterComponents = new BlobWriterComponents(dataFileWriter, azureBlobOutputStream, blockBlobAsyncClient); + allBlobWriterComponents.add(this.currentBlobWriterComponents); + LOG.info("Created new blob: {}", blobURL); + } + + private boolean willCurrentBlobExceedSize(byte[] encodedRecord) { + AzureBlobOutputStream azureBlobOutputStream = currentBlobWriterComponents.azureBlobOutputStream; + return (azureBlobOutputStream.getSize() + encodedRecord.length + DATAFILEWRITER_OVERHEAD) > maxBlobSize; + } + + private boolean willCurrentBlobExceedRecordLimit() { + return (recordsInCurrentBlob + 1) > maxRecordsPerBlob; + } + + /** + * Holds the components needed to write to an Azure Blob + * - including Avro's DataFileWriter, AzureBlobOutputStream and Azure's BlockBlobAsyncClient + */ + private class BlobWriterComponents { + final DataFileWriter dataFileWriter; + final AzureBlobOutputStream azureBlobOutputStream; + final BlockBlobAsyncClient blockBlobAsyncClient; + + public BlobWriterComponents(DataFileWriter dataFileWriter, AzureBlobOutputStream azureBlobOutputStream, + BlockBlobAsyncClient blockBlobAsyncClient) { + Preconditions.checkNotNull(dataFileWriter, "DataFileWriter can not be null when creating WriterComponents for an Azure Blob."); + Preconditions.checkNotNull(azureBlobOutputStream, "AzureBlobOutputStream can not be null when creating WriterComponents for an Azure Blob."); + Preconditions.checkNotNull(blockBlobAsyncClient, "BlockBlobAsyncClient can not be null when creating WriterComponents for an Azure Blob."); + this.dataFileWriter = dataFileWriter; + this.azureBlobOutputStream = azureBlobOutputStream; + this.blockBlobAsyncClient = blockBlobAsyncClient; + } + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriterFactory.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriterFactory.java new file mode 100644 index 0000000000..2510766312 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriterFactory.java @@ -0,0 +1,43 @@ +/* + * 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.system.azureblob.avro; + +import org.apache.samza.system.azureblob.compression.Compression; +import org.apache.samza.system.azureblob.producer.AzureBlobWriter; +import org.apache.samza.system.azureblob.producer.AzureBlobWriterFactory; +import org.apache.samza.system.azureblob.producer.AzureBlobWriterMetrics; +import com.azure.storage.blob.BlobContainerAsyncClient; +import java.io.IOException; +import java.util.concurrent.Executor; + + +public class AzureBlobAvroWriterFactory implements AzureBlobWriterFactory { + + /** + * {@inheritDoc} + */ + public AzureBlobWriter getWriterInstance(BlobContainerAsyncClient containerAsyncClient, String blobURL, + Executor blobUploadThreadPool, AzureBlobWriterMetrics metrics, + int maxBlockFlushThresholdSize, long flushTimeoutMs, Compression compression, boolean useRandomStringInBlobName, + long maxBlobSize, long maxMessagesPerBlob) throws IOException { + return new AzureBlobAvroWriter(containerAsyncClient, blobURL, blobUploadThreadPool, metrics, + maxBlockFlushThresholdSize, flushTimeoutMs, compression, useRandomStringInBlobName, maxBlobSize, maxMessagesPerBlob); + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java new file mode 100644 index 0000000000..2b25f96652 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java @@ -0,0 +1,305 @@ +/* + * 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.system.azureblob.avro; + +import com.azure.storage.blob.specialized.BlockBlobAsyncClient; +import com.google.common.annotations.VisibleForTesting; +import org.apache.samza.AzureException; +import org.apache.samza.system.azureblob.compression.Compression; +import org.apache.samza.system.azureblob.producer.AzureBlobWriterMetrics; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Base64; +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import reactor.core.publisher.Flux; + +/** + * This class extends {@link java.io.OutputStream} and uses {@link java.io.ByteArrayOutputStream} + * for caching the write calls till upload is not called. + * + * It asynchronously uploads the blocks and waits on them to finish at close. + * The blob is persisted at close. + * + * flush must be explicitly called before close. + * Any writes after a flush and before a close will be lost if no flush is called just before close. + * Once closed this object can not be used. + * + * releaseBuffer releases the underlying buffer i.e ByteArrayOutputStream which holds the data written until it is flushed. + * flush must be explicitly called prior to releaseBuffer else all data written + * since the beginning/previous flush will be lost. + * No data can be written after releaseBuffer, flush after releaseBuffer is a no-op + * and close must still be invoked to wait for all pending uploads to finish and persist the blob. + * releaseBuffer is optional and maybe called after its last flush and before close (which might happen much later), + * so as to reduce the overall memory footprint. close can not replace releaseBuffer as it is a blocking call. + * + * This library is thread safe. + */ +public class AzureBlobOutputStream extends OutputStream { + + private static final Logger LOG = LoggerFactory.getLogger(AzureBlobOutputStream.class); + private static final int MAX_ATTEMPT = 3; + private static final int MAX_BLOCKS_IN_AZURE_BLOB = 50000; + public static final String BLOB_RAW_SIZE_BYTES_METADATA = "rawSizeBytes"; + private final long flushTimeoutMs; + private final BlockBlobAsyncClient blobAsyncClient; + private final Executor blobThreadPool; + private Optional byteArrayOutputStream; + // All the block Names should be explicitly present in the blockList during CommitBlockList, + // even if stageBlock is a blocking call. + private final ArrayList blockList; + private final Set> pendingUpload = ConcurrentHashMap.newKeySet(); + private final int maxBlockFlushThresholdSize; + private final AzureBlobWriterMetrics metrics; + private final Compression compression; + + private volatile boolean isClosed = false; + private long totalUploadedBlockSize = 0; + private int blockNum; + + public AzureBlobOutputStream(BlockBlobAsyncClient blobAsyncClient, Executor blobThreadPool, AzureBlobWriterMetrics metrics, + long flushTimeoutMs, int maxBlockFlushThresholdSize, Compression compression) { + this(blobAsyncClient, blobThreadPool, metrics, flushTimeoutMs, maxBlockFlushThresholdSize, + new ByteArrayOutputStream(maxBlockFlushThresholdSize), compression); + } + + /** + * {@inheritDoc} + */ + @Override + public synchronized void write(int b) { + if (!byteArrayOutputStream.isPresent()) { + throw new IllegalStateException("Internal Buffer must have been released earlier for blob " + blobAsyncClient.getBlobUrl().toString()); + } + + if (byteArrayOutputStream.get().size() + 1 > maxBlockFlushThresholdSize) { + uploadBlockAsync(); + } + byteArrayOutputStream.get().write(b); + metrics.updateWriteByteMetrics(1); + } + + /** + * {@inheritDoc} + */ + @Override + public synchronized void write(byte[] b, int off, int len) { + if (!byteArrayOutputStream.isPresent()) { + throw new IllegalStateException("Internal Buffer must have been released earlier for blob " + blobAsyncClient.getBlobUrl().toString()); + } + + int remainingBytes = len; + int offset = off; + while (remainingBytes > 0) { + int bytesToWrite = Math.min(maxBlockFlushThresholdSize - byteArrayOutputStream.get().size(), remainingBytes); + byteArrayOutputStream.get().write(b, offset, bytesToWrite); + offset += bytesToWrite; + remainingBytes -= bytesToWrite; + if (byteArrayOutputStream.get().size() >= maxBlockFlushThresholdSize) { + uploadBlockAsync(); + } + } + metrics.updateWriteByteMetrics(len); + } + /** + * {@inheritDoc} + */ + @Override + public synchronized void flush() { + if (byteArrayOutputStream.isPresent()) { + uploadBlockAsync(); + } + } + + /** + * This api waits for all pending upload (stageBlock task) futures to finish. + * It then synchronously commits the list of blocks to persist the actual blob on storage. + * @throws IllegalStateException when + * - when closing an already closed stream + * @throws RuntimeException when + * - byteArrayOutputStream.close fails or + * - any of the pending uploads fails or + * - blob's commitBlockList fails + */ + @Override + public synchronized void close() { + if (isClosed) { + LOG.info("{}: already closed", blobAsyncClient.getBlobUrl().toString()); + return; + } + + LOG.info("{}: Close", blobAsyncClient.getBlobUrl().toString()); + try { + if (byteArrayOutputStream.isPresent()) { + byteArrayOutputStream.get().close(); + } + if (blockList.size() == 0) { + return; + } + CompletableFuture future = + CompletableFuture.allOf(pendingUpload.toArray(new CompletableFuture[0])); + + LOG.info("Closing blob: {} PendingUpload:{} ", blobAsyncClient.getBlobUrl().toString(), pendingUpload.size()); + + future.get((long) flushTimeoutMs, TimeUnit.MILLISECONDS); + LOG.info("For blob: {} committing blockList size:{}", blobAsyncClient.getBlobUrl().toString(), blockList.size()); + metrics.updateAzureCommitMetrics(); + Map blobMetadata = Collections.singletonMap(BLOB_RAW_SIZE_BYTES_METADATA, Long.toString(totalUploadedBlockSize)); + blobAsyncClient.commitBlockListWithResponse(blockList, null, blobMetadata, null, null).block(); + } catch (Exception e) { + String msg = String.format("Close blob %s failed with exception. Total pending sends %d", + blobAsyncClient.getBlobUrl().toString(), pendingUpload.size()); + throw new AzureException(msg, e); + } finally { + blockList.clear(); + pendingUpload.stream().forEach(future -> future.cancel(true)); + pendingUpload.clear(); + isClosed = true; + } + } + + /** + * Returns the size of the blob so far including data in the uploaded blocks and data currently in buffer. + * @return data written since the beginning + */ + public synchronized long getSize() { + return byteArrayOutputStream.isPresent() ? byteArrayOutputStream.get().size() + totalUploadedBlockSize : totalUploadedBlockSize; + } + + /** + * Releases the underlying buffer i.e; ByteArrayOutputStream. + * flush must be explicitly called prior to releaseBuffer else all data written + * since the beginning/previous flush will be lost. + * No data can be written after releaseBuffer, flush after releaseBuffer is a no-op + * and close must still be invoked to wait for all pending uploads to finish and persist the blob. + * This is optional and can be used to release memory. + * @throws IOException if ByteArrayOutputStream.close fails + */ + public synchronized void releaseBuffer() throws IOException { + if (byteArrayOutputStream.isPresent()) { + byteArrayOutputStream.get().close(); + byteArrayOutputStream = Optional.empty(); + LOG.info("Internal buffer has been released for blob " + blobAsyncClient.getBlobUrl().toString() + + ". Writes are no longer entertained."); + } + } + + @VisibleForTesting + AzureBlobOutputStream(BlockBlobAsyncClient blobAsyncClient, Executor blobThreadPool, AzureBlobWriterMetrics metrics, + long flushTimeoutMs, int maxBlockFlushThresholdSize, + ByteArrayOutputStream byteArrayOutputStream, Compression compression) { + this.byteArrayOutputStream = Optional.of(byteArrayOutputStream); + this.blobAsyncClient = blobAsyncClient; + blockList = new ArrayList<>(); + blockNum = 0; + this.blobThreadPool = blobThreadPool; + this.flushTimeoutMs = flushTimeoutMs; + this.maxBlockFlushThresholdSize = maxBlockFlushThresholdSize; + this.metrics = metrics; + this.compression = compression; + } + + /** + * This api will async upload the outputstream into block using stageBlocks, + * reint outputstream + * and add the operation to future. + * @throws RuntimeException when + * - blob's stageBlock fails after MAX_ATTEMPTs + * - number of blocks exceeds MAX_BLOCKS_IN_AZURE_BLOB + */ + private synchronized void uploadBlockAsync() { + if (!byteArrayOutputStream.isPresent()) { + return; + } + long size = byteArrayOutputStream.get().size(); + if (size == 0) { + return; + } + LOG.info("Blob: {} uploadBlock. Size:{}", blobAsyncClient.getBlobUrl().toString(), size); + + // Azure sdk requires block Id to be encoded and all blockIds of a blob to be of the same length + // also, a block blob can have upto 50,000 blocks, hence using a 5 digit block id. + String blockId = String.format("%05d", blockNum); + String blockIdEncoded = Base64.getEncoder().encodeToString(blockId.getBytes()); + blockList.add(blockIdEncoded); + byte[] localByte = byteArrayOutputStream.get().toByteArray(); + byteArrayOutputStream.get().reset(); + totalUploadedBlockSize += localByte.length; + + CompletableFuture future = CompletableFuture.runAsync(new Runnable() { + // call async stageblock and add to future + @Override + public void run() { + int attemptCount = 0; + byte[] compressedLocalByte = compression.compress(localByte); + int blockSize = compressedLocalByte.length; + + while (attemptCount < MAX_ATTEMPT) { + try { + ByteBuffer outputStream = ByteBuffer.wrap(compressedLocalByte, 0, blockSize); + metrics.updateCompressByteMetrics(blockSize); + LOG.info("{} Upload block start for blob: {} for block size:{}.", blobAsyncClient.getBlobUrl().toString(), blockId, blockSize); + metrics.updateAzureUploadMetrics(); + // StageBlock generates exception on Failure. + blobAsyncClient.stageBlock(blockIdEncoded, Flux.just(outputStream), blockSize).block(); + break; + } catch (Exception e) { + attemptCount += 1; + String msg = "Upload block for blob: " + blobAsyncClient.getBlobUrl().toString() + + " failed for blockid: " + blockId + " due to exception. AttemptCount: " + attemptCount; + LOG.error(msg, e); + if (attemptCount == MAX_ATTEMPT) { + throw new AzureException("Exceeded number of attempts. Max attempts is: " + MAX_ATTEMPT, e); + } + } + } + } + }, blobThreadPool); + + pendingUpload.add(future); + + future.handle((aVoid, throwable) -> { + if (throwable == null) { + LOG.info("Upload block for blob: {} with blockid: {} finished.", blobAsyncClient.getBlobUrl().toString(), blockId); + pendingUpload.remove(future); + return aVoid; + } else { + throw new AzureException("Blob upload failed for blob " + blobAsyncClient.getBlobUrl().toString() + + " and block with id: " + blockId, throwable); + } + }); + + blockNum += 1; + if (blockNum >= MAX_BLOCKS_IN_AZURE_BLOB) { + throw new AzureException("Azure blob only supports 50000 blocks in a blob. Current number of blocks is " + blockNum); + } + } +} \ No newline at end of file diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/Compression.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/Compression.java new file mode 100644 index 0000000000..caaab178d2 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/Compression.java @@ -0,0 +1,41 @@ +/* + * 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.system.azureblob.compression; + +/** + * Interface for compression. + */ +public interface Compression { + /** + * Compresses a byte[]. + * Depending on the type of compression, the compressed byte[] might be larger than input for small inputs. + * For example: Gzip compression of an 0 length byte array yields byte[24] due to CRC and other metadata it adds. + * @param input byte[] to be compressed + * @return compressed byte[]. + */ + byte[] compress(byte[] input); + + /** + * The file extension to be used for the particular compression. + * For example: .gz for GZIP compression and "" for no compression. + * @return + */ + String getFileExtension(); +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionFactory.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionFactory.java new file mode 100644 index 0000000000..0f738fcc09 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionFactory.java @@ -0,0 +1,43 @@ +/* + * 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.system.azureblob.compression; + +/** + * This factory instantiates the appropriate implementation of + * {@link org.apache.samza.system.azureblob.compression.Compression} + * based on the {@link org.apache.samza.system.azureblob.compression.CompressionType}. + */ +public class CompressionFactory { + private final static CompressionFactory COMPRESSION_FACTORY_INSTANCE = new CompressionFactory(); + private CompressionFactory() {} + + public static CompressionFactory getInstance() { + return COMPRESSION_FACTORY_INSTANCE; + } + + public Compression getCompression(CompressionType compressionType) { + switch (compressionType) { + case NONE: return new NoneCompression(); + case GZIP: return new GzipCompression(); + default: throw new IllegalArgumentException("Unknown compression name: " + compressionType.name()); + } + + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionType.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionType.java new file mode 100644 index 0000000000..6b8c176702 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionType.java @@ -0,0 +1,31 @@ +/* + * 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.system.azureblob.compression; + +/** + * The compression type to use. + * Supported compression types: + * - None: no compression is applied + * - GZIP: gzip compression is applied + */ +public enum CompressionType { + NONE, GZIP +} + diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/GzipCompression.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/GzipCompression.java new file mode 100644 index 0000000000..d2d2f2997d --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/GzipCompression.java @@ -0,0 +1,75 @@ +/* + * 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.system.azureblob.compression; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.zip.GZIPOutputStream; +import org.apache.commons.lang3.Validate; +import org.apache.samza.SamzaException; + + +/** + * This class implements the {@link org.apache.samza.system.azureblob.compression.Compression}. + * It uses GZIPOutputStream to compress the given byte[]. + * The file extension for to be used for this compressed data is ".gz" + */ +public class GzipCompression implements Compression { + /** + * {@inheritDoc} + * @throws SamzaException if compression fails + */ + @Override + public byte[] compress(byte[] input) { + Validate.notNull(input, "Input for compression is null"); + + ByteArrayOutputStream byteArrayOutputStream = null; + GZIPOutputStream gzipOutputStream = null; + try { + byteArrayOutputStream = new ByteArrayOutputStream(input.length); + gzipOutputStream = new GZIPOutputStream(byteArrayOutputStream); + gzipOutputStream.write(input); + gzipOutputStream.close(); + return byteArrayOutputStream.toByteArray(); + + } catch (IOException e) { + throw new SamzaException("Failed to compress.", e); + } finally { + try { + if (gzipOutputStream != null) { + gzipOutputStream.close(); + } + if (byteArrayOutputStream != null) { + byteArrayOutputStream.close(); + } + } catch (Exception e) { + throw new SamzaException("Failed to close output streams during compression.", e); + } + } + } + + /** + * {@inheritDoc} + */ + @Override + public String getFileExtension() { + return ".gz"; + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/NoneCompression.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/NoneCompression.java new file mode 100644 index 0000000000..c76322cc52 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/NoneCompression.java @@ -0,0 +1,37 @@ +/* + * 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.system.azureblob.compression; + +/** + * This class implements the {@link org.apache.samza.system.azureblob.compression.Compression}. + * It does not compress the input and returns it as is. + * There is no file extension. + */ +public class NoneCompression implements Compression { + @Override + public byte[] compress(byte[] input) { + return input; + } + + @Override + public String getFileExtension() { + return ""; + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducer.java new file mode 100644 index 0000000000..a2fa2acb05 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducer.java @@ -0,0 +1,518 @@ +/* + * 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.system.azureblob.producer; + +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.ProxyOptions; +import com.azure.core.http.netty.NettyAsyncHttpClientBuilder; +import com.azure.core.http.policy.HttpLogDetailLevel; +import com.azure.core.http.policy.HttpLogOptions; +import com.azure.core.util.Configuration; +import com.azure.storage.blob.BlobContainerAsyncClient; +import com.azure.storage.blob.BlobServiceAsyncClient; +import com.azure.storage.blob.BlobServiceClientBuilder; +import com.azure.storage.blob.models.BlobErrorCode; +import com.azure.storage.blob.models.BlobStorageException; +import com.azure.storage.blob.models.SkuName; +import com.azure.storage.common.StorageSharedKeyCredential; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.samza.system.azureblob.AzureBlobConfig; +import org.apache.samza.system.azureblob.compression.CompressionFactory; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.system.OutgoingMessageEnvelope; +import org.apache.samza.system.SystemProducer; +import org.apache.samza.system.SystemProducerException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * AzureBlob system producer to send messages to Azure Blob Storage. + * This system producer is thread safe. + * For different sources: sends/flushes can happen in parallel. + * For same source: It supports sends in parallel. flushes are exclusive. + * + * + * Azure Blob Storage has a 3 level hierarchy: an Azure account contains multiple containers (akin to directories + * in a file system) and each container has multiple blobs (akin to files). + * + * Azure Container: System name maps to the name of Azure container. + * An instance of a system producer writes to a single Azure container considering the container as a system. + * + * Azure Blob: For a given stream-partition pair, a blob is created with name stream/partition/timestamp-randomString. + * The stream and partition are extracted from the SSP of OutgoingMessageEnvelope in send(). + * Blob is started when the first message for that stream-partition is sent by a source + * and closed during flush for that source. + * Subsequent sends by the source to the same stream-partition will create a new blob with a different timestamp. + * Thus, timestamp corresponds to writer creation time i.e; the first send for source-SSP + * or first send after a flush for the source. + * If max blob size or record limit are configured, then a new blob is started when limits exceed. + * + * A random string is used as a suffix in the blob name to prevent collisions: + * - if two system producers are writing to the same SSP. + * - if two sources send to the same SSP. + * + * Lifecycle of the system producer is shown below. All sources have to be registered before starting the producer. + * Several messages can be sent by a source via send(source, envelope). This can be followed by a flush(source) or stop() + * After flush(source), more messages can be sent for that source and other sources as well. stop() internally calls + * flush(source) for all the sources registered. After stop(), no calls to send and flush are allowed. + * + * + * ┌──────────────────────────────┐ + * │ │ + * ▼ │ + * Lifecycle: register(source) ────────▶ start() ──────▶ send(source, envelope) ──────▶ flush(source) ──────▶ stop() + * [multiple times │ ▲ │ ▲ + * for └────┘ └─────────────────────────────────────┘ + * multiple sources] + * + * This SystemProducer does not open up the envelopes sent through it. It is the responsibility of the user of this + * SystemProducer to ensure the envelopes are valid and a correct writer has been chosen by wiring up the + * writer factory config. + * + */ +public class AzureBlobSystemProducer implements SystemProducer { + + private static final Logger LOG = LoggerFactory.getLogger(AzureBlobSystemProducer.class.getName()); + + private static final String BLOB_NAME_PREFIX = "%s"; + private static final String BLOB_NAME_PARTITION_PREFIX = "%s/%s"; + + private static final String AZURE_URL = "https://%s.blob.core.windows.net"; + + private static final int PREMIUM_MAX_BLOCK_SIZE = 100 * 1024 * 1024; // 100MB + private static final int STANDARD_MAX_BLOCK_SIZE = 4 * 1024 * 1024; // 4MB + + private BlobContainerAsyncClient containerAsyncClient; + private final String systemName; + private final AzureBlobConfig config; + + // Map of writers indexed first by sourceName and then by (streamName, partitionName) or just streamName if partition key does not exist. + private final Map> writerMap; + private final AzureBlobWriterFactory writerFactory; + + private final int blockFlushThresholdSize; + private final long flushTimeoutMs; + private final long closeTimeout; + private final ThreadPoolExecutor asyncBlobThreadPool; + + private volatile boolean isStarted = false; + private volatile boolean isStopped = false; + + private final AzureBlobSystemProducerMetrics metrics; + + private final Map sourceWriterCreationLockMap = new ConcurrentHashMap<>(); + private final Map sourceSendFlushLockMap = new ConcurrentHashMap<>(); + + public AzureBlobSystemProducer(String systemName, AzureBlobConfig config, MetricsRegistry metricsRegistry) { + Preconditions.checkNotNull(systemName, "System name can not be null when creating AzureBlobSystemProducer"); + Preconditions.checkNotNull(config, "Config can not be null when creating AzureBlobSystemProducer"); + Preconditions.checkNotNull(metricsRegistry, "Metrics registry can not be null when creating AzureBlobSystemProducer"); + + // Azure logs do not show without this property set + System.setProperty(Configuration.PROPERTY_AZURE_LOG_LEVEL, "1"); + this.systemName = systemName; + this.config = config; + + String writerFactoryClassName = this.config.getAzureBlobWriterFactoryClassName(this.systemName); + try { + this.writerFactory = (AzureBlobWriterFactory) Class.forName(writerFactoryClassName).newInstance(); + } catch (Exception e) { + throw new SystemProducerException("Could not create writer factory with name " + writerFactoryClassName, e); + } + this.flushTimeoutMs = this.config.getFlushTimeoutMs(this.systemName); + this.closeTimeout = this.config.getCloseTimeoutMs(this.systemName); + this.blockFlushThresholdSize = this.config.getMaxFlushThresholdSize(this.systemName); + int asyncBlobThreadPoolCount = this.config.getAzureBlobThreadPoolCount(this.systemName); + int blockingQueueSize = this.config.getBlockingQueueSize(this.systemName); + + LOG.info("SystemName: {} block flush size:{}", systemName, this.blockFlushThresholdSize); + LOG.info("SystemName: {} thread count:{}", systemName, asyncBlobThreadPoolCount); + + BlockingQueue + linkedBlockingDeque = new LinkedBlockingDeque<>(blockingQueueSize); + + this.asyncBlobThreadPool = + new ThreadPoolExecutor(asyncBlobThreadPoolCount, asyncBlobThreadPoolCount, 60, + TimeUnit.SECONDS, linkedBlockingDeque, new ThreadPoolExecutor.CallerRunsPolicy()); + + this.writerMap = new ConcurrentHashMap<>(); + + this.metrics = new AzureBlobSystemProducerMetrics(systemName, config.getAzureAccountName(systemName), metricsRegistry); + } + + /** + * {@inheritDoc} + * @throws SystemProducerException + */ + @Override + public synchronized void start() { + if (isStarted) { + LOG.warn("Attempting to start an already started producer."); + return; + } + + String accountName = config.getAzureAccountName(systemName); + String accountKey = config.getAzureAccountKey(systemName); + + setupAzureContainer(accountName, accountKey); + + LOG.info("Starting producer."); + isStarted = true; + } + + /** + * {@inheritDoc} + * @throws SystemProducerException + */ + @Override + public synchronized void stop() { + if (!isStarted) { + LOG.warn("Attempting to stop a producer that was not started."); + return; + } + + if (isStopped) { + LOG.warn("Attempting to stop an already stopped producer."); + return; + } + + try { + writerMap.forEach((source, sourceWriterMap) -> flush(source)); + asyncBlobThreadPool.shutdown(); + isStarted = false; + } catch (Exception e) { + throw new SystemProducerException("Stop failed with exception.", e); + } finally { + writerMap.clear(); + isStopped = true; + } + } + + /** + * {@inheritDoc} + * @throws SystemProducerException + */ + @Override + public synchronized void register(String source) { + LOG.info("Registering source {}", source); + if (isStarted) { + throw new SystemProducerException("Cannot register once the producer is started."); + } + if (writerMap.containsKey(source)) { + // source already registered => writerMap and metrics have entries for the source + LOG.warn("Source: {} already registered", source); + return; + } + writerMap.put(source, new ConcurrentHashMap<>()); + sourceWriterCreationLockMap.put(source, new Object()); + sourceSendFlushLockMap.put(source, new ReentrantReadWriteLock()); + metrics.register(source); + } + + /** + * Multi-threading and thread-safety: + * + * From Samza usage of SystemProducer: + * The lifecycle of SystemProducer shown above is consistent with most use cases within Samza (with the exception of + * Coordinator stream store/producer and KafkaCheckpointManager). + * A single parent thread creates the SystemProducer, registers all sources and starts it before handing it + * to multiple threads for use (send and flush). Finally, the single parent thread stops the producer. + * The most frequent operations on a SystemProducer are send and flush while register, start and stop are one-time operations. + * + * Based on this usage pattern: to provide multi-threaded support and improve throughput of this SystemProducer, + * multiple sends and flushes need to happen in parallel. However, the following rules are needed to ensure + * o data loss and data consistency. + * 1. sends can happen in parallel for same source or different sources. + * 2. send and flush for the same source can not happen in parallel. Although, the AzureBlobWriter is thread safe, + * interleaving write and flush and close operations of a writer can lead to data loss if a write happens between flush and close. + * There are other scenarios such as issuing a write to the writer after close and so on. + * 3. writer creation for the same writer key (SSP) can not happen in parallel - for the reason that multiple + * writers could get created with only one being retained but all being used and GCed after a send, leading to data loss. + * + * These 3 rules are achieved by using a per source ReadWriteLock to allow sends in parallel but guarantee exclusivity for flush. + * Additionally, a per source lock is used to ensure writer creation is in a critical section. + * + * Concurrent access to shared objects as follows: + * 1. AzureBlobWriters is permitted as long as there are no interleaving of operations for a writer. + * If multiple operations of writer (as in flush) then make it synchronized. + * 2. ConcurrentHashMaps (esp writerMap per source) get and put - disallow interleaving by doing put and clear under locks. + * 3. WriterFactory and Metrics are thread-safe. WriterFactory is stateless while Metrics' operations interleaving + * is thread-safe too as they work on different counters. + * The above locking mechanisms ensure thread-safety. + * {@inheritDoc} + * @throws SystemProducerException + */ + @Override + public void send(String source, OutgoingMessageEnvelope messageEnvelope) { + if (!isStarted) { + throw new SystemProducerException("Trying to send before producer has started."); + } + + if (isStopped) { + throw new SystemProducerException("Sending after producer has been stopped."); + } + + ReadWriteLock lock = sourceSendFlushLockMap.get(source); + if (lock == null) { + throw new SystemProducerException("Attempting to send to source: " + source + " but it was not registered"); + } + lock.readLock().lock(); + try { + AzureBlobWriter writer = getOrCreateWriter(source, messageEnvelope); + writer.write(messageEnvelope); + metrics.updateWriteMetrics(source); + } catch (Exception e) { + metrics.updateErrorMetrics(source); + Object partitionKey = getPartitionKey(messageEnvelope); + String msg = "Send failed for source: " + source + ", system: " + systemName + + ", stream: " + messageEnvelope.getSystemStream().getStream() + + ", partitionKey: " + ((partitionKey != null) ? partitionKey : "null"); + throw new SystemProducerException(msg, e); + } finally { + lock.readLock().unlock(); + } + } + + /** + * {@inheritDoc} + * @throws SystemProducerException + */ + @Override + public void flush(String source) { + if (!isStarted) { + throw new SystemProducerException("Trying to flush before producer has started."); + } + + if (isStopped) { + throw new SystemProducerException("Flushing after producer has been stopped."); + } + + ReadWriteLock lock = sourceSendFlushLockMap.get(source); + if (lock == null) { + throw new SystemProducerException("Attempting to flush source: " + source + " but it was not registered"); + } + lock.writeLock().lock(); + Map sourceWriterMap = writerMap.get(source); + try { + // first flush all the writers + // then close and remove all the writers + flushWriters(sourceWriterMap); + closeWriters(source, sourceWriterMap); + } catch (Exception e) { + metrics.updateErrorMetrics(source); + throw new SystemProducerException("Flush failed for system:" + systemName + " and source: " + source, e); + } finally { + sourceWriterMap.clear(); + lock.writeLock().unlock(); + } + } + + @VisibleForTesting + void setupAzureContainer(String accountName, String accountKey) { + try { + // Use your Azure Blob Storage account's name and key to create a credential object to access your account. + StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey); + + HttpClient httpClient; + if (config.getUseProxy(systemName)) { + LOG.info("HTTP Proxy setup for AzureBlob pipeline"); + httpClient = new NettyAsyncHttpClientBuilder() + .proxy(new ProxyOptions(ProxyOptions.Type.HTTP, + new InetSocketAddress(config.getAzureProxyHostname(systemName), config.getAzureProxyPort(systemName)))).build(); + } else { + httpClient = HttpClient.createDefault(); + } + + // From the Azure portal, get your Storage account blob service AsyncClient endpoint. + String endpoint = String.format(Locale.ROOT, AZURE_URL, accountName); + + HttpLogOptions httpLogOptions = new HttpLogOptions(); + httpLogOptions.setLogLevel(HttpLogDetailLevel.BASIC); + BlobServiceAsyncClient storageClient = + new BlobServiceClientBuilder() + .httpLogOptions(httpLogOptions) + .endpoint(endpoint) + .credential(credential) + .httpClient(httpClient) + .buildAsyncClient(); + + + SkuName accountType = storageClient.getAccountInfo().block().getSkuName(); + long flushThresholdSize = config.getMaxFlushThresholdSize(systemName); + boolean isPremiumAccount = SkuName.PREMIUM_LRS == accountType; + if (isPremiumAccount && flushThresholdSize > PREMIUM_MAX_BLOCK_SIZE) { // 100 MB + throw new SystemProducerException("Azure storage account with name: " + accountName + + " is a premium account and can only handle upto " + PREMIUM_MAX_BLOCK_SIZE + " threshold size. Given flush threshold size is " + + flushThresholdSize); + } else if (!isPremiumAccount && flushThresholdSize > STANDARD_MAX_BLOCK_SIZE) { // STANDARD account + throw new SystemProducerException("Azure storage account with name: " + accountName + + " is a standard account and can only handle upto " + STANDARD_MAX_BLOCK_SIZE + " threshold size. Given flush threshold size is " + + flushThresholdSize); + } + + containerAsyncClient = storageClient.getBlobContainerAsyncClient(systemName); + + // Only way to check if container exists or not is by creating it and look for failure/success. + createContainerIfNotExists(containerAsyncClient); + } catch (Exception e) { + metrics.updateAzureContainerMetrics(); + throw new SystemProducerException("Failed to set up Azure container for SystemName: " + systemName, e); + } + } + + /** + * // find the writer in the writerMap else create one + * @param source for which to find/create the writer + * @param messageEnvelope to fetch the schema from if writer needs to be created + * @return an AzureBlobWriter object + */ + @VisibleForTesting + AzureBlobWriter getOrCreateWriter(String source, OutgoingMessageEnvelope messageEnvelope) { + String writerMapKey; + String blobURLPrefix; + String partitionKey = getPartitionKey(messageEnvelope); + // using most significant bits in UUID (8 digits) to avoid collision in blob names + if (partitionKey == null) { + writerMapKey = messageEnvelope.getSystemStream().getStream(); + blobURLPrefix = String.format(BLOB_NAME_PREFIX, messageEnvelope.getSystemStream().getStream()); + } else { + writerMapKey = messageEnvelope.getSystemStream().getStream() + "/" + partitionKey; + blobURLPrefix = String.format(BLOB_NAME_PARTITION_PREFIX, messageEnvelope.getSystemStream().getStream(), partitionKey); + } + Map sourceWriterMap = writerMap.get(source); + if (sourceWriterMap == null) { + throw new SystemProducerException("Attempting to send to source: " + source + " but it is not registered"); + } + AzureBlobWriter writer = sourceWriterMap.get(writerMapKey); + if (writer == null) { + synchronized (sourceWriterCreationLockMap.get(source)) { + writer = sourceWriterMap.get(writerMapKey); + if (writer == null) { + AzureBlobWriterMetrics writerMetrics = + new AzureBlobWriterMetrics(metrics.getAggregateMetrics(), metrics.getSystemMetrics(), metrics.getSourceMetrics(source)); + writer = createNewWriter(blobURLPrefix, writerMetrics); + sourceWriterMap.put(writerMapKey, writer); + } + } + } + return writer; + } + + private void createContainerIfNotExists(BlobContainerAsyncClient containerClient) { + try { + containerClient.create().block(); + } catch (BlobStorageException e) { + //StorageErrorCode defines constants corresponding to all error codes returned by the service. + if (e.getErrorCode() == BlobErrorCode.RESOURCE_NOT_FOUND) { + HttpResponse response = e.getResponse(); + LOG.error("Error creating the container url " + containerClient.getBlobContainerUrl().toString() + " with status code: " + response.getStatusCode(), e); + } else if (e.getErrorCode() == BlobErrorCode.CONTAINER_BEING_DELETED) { + LOG.error("Container is being deleted. Container URL is: " + containerClient.getBlobContainerUrl().toString(), e); + } else if (e.getErrorCode() == BlobErrorCode.CONTAINER_ALREADY_EXISTS) { + return; + } + throw e; + } + } + + private String getPartitionKey(OutgoingMessageEnvelope messageEnvelope) { + Object partitionKey = messageEnvelope.getPartitionKey(); + if (partitionKey == null || !(partitionKey instanceof String)) { + return null; + } + return (String) partitionKey; + } + + private void flushWriters(Map sourceWriterMap) { + sourceWriterMap.forEach((stream, writer) -> { + try { + LOG.info("Flushing topic:{}", stream); + writer.flush(); + } catch (IOException e) { + throw new SystemProducerException("Close failed for topic " + stream, e); + } + }); + } + + private void closeWriters(String source, Map sourceWriterMap) throws Exception { + Set> pendingClose = ConcurrentHashMap.newKeySet(); + try { + sourceWriterMap.forEach((stream, writer) -> { + LOG.info("Closing topic:{}", stream); + CompletableFuture future = CompletableFuture.runAsync(new Runnable() { + @Override + public void run() { + try { + writer.close(); + } catch (IOException e) { + throw new SystemProducerException("Close failed for topic " + stream, e); + } + } + }, asyncBlobThreadPool); + pendingClose.add(future); + future.handle((aVoid, throwable) -> { + sourceWriterMap.remove(writer); + if (throwable != null) { + throw new SystemProducerException("Close failed for topic " + stream, throwable); + } else { + LOG.info("Blob close finished for stream " + stream); + return aVoid; + } + }); + }); + CompletableFuture future = CompletableFuture.allOf(pendingClose.toArray(new CompletableFuture[0])); + LOG.info("Flush source: {} has pending closes: {} ", source, pendingClose.size()); + future.get((long) closeTimeout, TimeUnit.MILLISECONDS); + } finally { + pendingClose.clear(); + } + } + + @VisibleForTesting + AzureBlobWriter createNewWriter(String blobURL, AzureBlobWriterMetrics writerMetrics) { + try { + return writerFactory.getWriterInstance(containerAsyncClient, blobURL, asyncBlobThreadPool, writerMetrics, + blockFlushThresholdSize, flushTimeoutMs, + CompressionFactory.getInstance().getCompression(config.getCompressionType(systemName)), + config.getSuffixRandomStringToBlobName(systemName), + config.getMaxBlobSize(systemName), + config.getMaxMessagesPerBlob(systemName)); + } catch (Exception e) { + throw new RuntimeException("Failed to create a writer for the producer.", e); + } + } + +} \ No newline at end of file diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducerMetrics.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducerMetrics.java new file mode 100644 index 0000000000..6629e68be4 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducerMetrics.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.samza.system.azureblob.producer; + +import org.apache.samza.system.azureblob.AzureBlobBasicMetrics; +import java.util.HashMap; +import java.util.Map; +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.MetricsRegistry; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * This class holds all the metrics for a {@link org.apache.samza.system.azureblob.producer.AzureBlobSystemProducer}. + * It maintains: aggregate metrics, system level metrics and source metrics for all the sources that register and + * send through the SystemProducer. It has a map for holding the metrics of all sources. + * + * Apart from the basic metrics for each group, this class also holds metrics for Azure container creation errors. + */ +public class AzureBlobSystemProducerMetrics { + private static final Logger LOG = LoggerFactory.getLogger(AzureBlobSystemProducerMetrics.class.getName()); + + protected static final String AZURE_CONTAINER_ERROR = "azureContainerError"; + protected static final String AGGREGATE = "aggregate"; + protected static final String SYSTEM_METRIC_FORMAT = "%s_%s"; + + private final MetricsRegistry metricsRegistry; + private final Map sourceMetricsMap; + private final AzureBlobBasicMetrics aggregateMetrics; + private final AzureBlobBasicMetrics systemMetrics; + private final Counter aggregateAzureContainerErrorMetrics; + private final Counter systemAzureContainerErrorMetrics; + + private final String systemName; + private final String accountName; + + public AzureBlobSystemProducerMetrics(String systemName, String accountName, MetricsRegistry metricsRegistry) { + this.metricsRegistry = metricsRegistry; + this.systemName = systemName; + this.accountName = accountName; + + sourceMetricsMap = new HashMap<>(); + aggregateMetrics = new AzureBlobBasicMetrics(AGGREGATE, metricsRegistry); + systemMetrics = new AzureBlobBasicMetrics(String.format(SYSTEM_METRIC_FORMAT, accountName, systemName), metricsRegistry); + aggregateAzureContainerErrorMetrics = metricsRegistry.newCounter(AGGREGATE, AZURE_CONTAINER_ERROR); + systemAzureContainerErrorMetrics = metricsRegistry.newCounter(String.format(SYSTEM_METRIC_FORMAT, accountName, systemName), AZURE_CONTAINER_ERROR); + } + + /** + * Adds a AzureBlobBasicMetrics object for the source being registered with the SystemProducer. + * @param source to be registered. + */ + public void register(String source) { + if (systemName.equals(source)) { + // source is the same as the system name. creating counters for source name will double count metrics + LOG.warn("Source:{} is the same as the system name.", source); + return; + } + sourceMetricsMap.putIfAbsent(source, new AzureBlobBasicMetrics(source, metricsRegistry)); + } + + /** + * Increments the error metrics counters of aggregate, system and the source by 1. + * @param source for which the error occurred. + */ + public void updateErrorMetrics(String source) { + AzureBlobBasicMetrics sourceMetrics = sourceMetricsMap.get(source); + if (sourceMetrics != null) { + sourceMetrics.updateErrorMetrics(); + } + incrementErrorMetrics(); + } + + /** + * Increments the write metrics counters of aggregate, system and the source by 1. + * Write metrics is for number of messages successfully written to the source. + * @param source for which the message was sent. + */ + public void updateWriteMetrics(String source) { + AzureBlobBasicMetrics sourceMetrics = sourceMetricsMap.get(source); + if (sourceMetrics != null) { + sourceMetrics.updateWriteMetrics(); + } + incrementWriteMetrics(); + } + + /** + * Increments the Azure container creation error metrics by 1. + */ + public void updateAzureContainerMetrics() { + aggregateAzureContainerErrorMetrics.inc(); + systemAzureContainerErrorMetrics.inc(); + } + + public AzureBlobBasicMetrics getAggregateMetrics() { + return aggregateMetrics; + } + + public AzureBlobBasicMetrics getSystemMetrics() { + return systemMetrics; + } + + public AzureBlobBasicMetrics getSourceMetrics(String source) { + return sourceMetricsMap.get(source); + } + + private void incrementWriteMetrics() { + aggregateMetrics.updateWriteMetrics(); + systemMetrics.updateWriteMetrics(); + } + + private void incrementErrorMetrics() { + aggregateMetrics.updateErrorMetrics(); + systemMetrics.updateErrorMetrics(); + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriter.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriter.java new file mode 100644 index 0000000000..f7fd4220bd --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriter.java @@ -0,0 +1,52 @@ +/* + * 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.system.azureblob.producer; + +import java.io.IOException; +import org.apache.samza.system.OutgoingMessageEnvelope; + +/** + * {@link org.apache.samza.system.azureblob.producer.AzureBlobSystemProducer} + * uses an AzureBlobWriter to write messages to Azure Blob Storage. + * + * Implementation is expected to be thread-safe. + */ +public interface AzureBlobWriter { + /** + * Write the given {@link org.apache.samza.system.OutgoingMessageEnvelope} to the blob opened. + * @param ome message to be written + */ + void write(OutgoingMessageEnvelope ome) throws IOException; + + /** + * Asynchronously upload the messages written as a block. + * After this the messages written will go as a new block. + */ + void flush() throws IOException; + + /** + * Close the writer and all of its underlying components. + * At the end of close, all the messages sent to the writer should be persisted in a blob. + * flush should be called explicitly before close. + * It is not the responsibility of close to upload blocks. + * After close, no other operations can be performed. + */ + void close() throws IOException; +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriterFactory.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriterFactory.java new file mode 100644 index 0000000000..87ca5a6a48 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriterFactory.java @@ -0,0 +1,44 @@ +/* + * 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.system.azureblob.producer; + +import org.apache.samza.system.azureblob.compression.Compression; +import com.azure.storage.blob.BlobContainerAsyncClient; +import java.io.IOException; +import java.util.concurrent.Executor; + + +public interface AzureBlobWriterFactory { + /** + * creates an instance of AzureBlobWriter. + * @param containerAsyncClient Azure container client + * @param blobURL Azure blob url + * @param blobUploadThreadPool thread pool to be used by writer for uploading + * @param metrics metrics to measure the number of bytes written by writer + * @param maxBlockFlushThresholdSize threshold at which to upload + * @param flushTimeoutMs timeout after which the flush is abandoned + * @return AzureBlobWriter instance + * @throws IOException if writer creation fails + */ + AzureBlobWriter getWriterInstance(BlobContainerAsyncClient containerAsyncClient, String blobURL, + Executor blobUploadThreadPool, AzureBlobWriterMetrics metrics, + int maxBlockFlushThresholdSize, long flushTimeoutMs, Compression compression, boolean useRandomStringInBlobName, + long maxBlobSize, long maxMessagesPerBlob) throws IOException; +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriterMetrics.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriterMetrics.java new file mode 100644 index 0000000000..5605fc6408 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriterMetrics.java @@ -0,0 +1,81 @@ +/* + * 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.system.azureblob.producer; + +import org.apache.samza.system.azureblob.AzureBlobBasicMetrics; + +/** + * This class holds the basic metrics object of type {@link org.apache.samza.system.azureblob.AzureBlobBasicMetrics} + * for each group to be tracked. + * It maintains 3 groups - aggregate metrics, system-level metrics and source metrics. + * + * This metrics is used by the AzureBlobWriter to measure the number of bytes written by the writer. + */ +public class AzureBlobWriterMetrics { + private final AzureBlobBasicMetrics systemMetrics; + private final AzureBlobBasicMetrics aggregateMetrics; + private final AzureBlobBasicMetrics sourceMetrics; + + public AzureBlobWriterMetrics(AzureBlobBasicMetrics systemMetrics, AzureBlobBasicMetrics aggregateMetrics, + AzureBlobBasicMetrics sourceMetrics) { + this.systemMetrics = systemMetrics; + this.aggregateMetrics = aggregateMetrics; + this.sourceMetrics = sourceMetrics; + } + + /** + * Increments the write byte metrics counters of all the groups by the number of bytes written. + * @param dataLength number of bytes written. + */ + public void updateWriteByteMetrics(long dataLength) { + systemMetrics.updateWriteByteMetrics(dataLength); + aggregateMetrics.updateWriteByteMetrics(dataLength); + sourceMetrics.updateWriteByteMetrics(dataLength); + } + + /** + * Increments the compress byte metrics counters of all the groups by the number of compressed bytes written. + * @param dataLength number of bytes written. + */ + public void updateCompressByteMetrics(long dataLength) { + systemMetrics.updateCompressByteMetrics(dataLength); + aggregateMetrics.updateCompressByteMetrics(dataLength); + sourceMetrics.updateCompressByteMetrics(dataLength); + } + + /** + * Increments the azure block upload metrics counters of all the groups + */ + public void updateAzureUploadMetrics() { + systemMetrics.updateAzureUploadMetrics(); + aggregateMetrics.updateAzureUploadMetrics(); + sourceMetrics.updateAzureUploadMetrics(); + } + + /** + * Increments the azure blob commit metrics counters of all the groups + */ + public void updateAzureCommitMetrics() { + systemMetrics.updateAzureCommitMetrics(); + aggregateMetrics.updateAzureCommitMetrics(); + sourceMetrics.updateAzureCommitMetrics(); + } + +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java new file mode 100644 index 0000000000..b4ef4b4e95 --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java @@ -0,0 +1,528 @@ +/* + * 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.system.azureblob.avro; + +import com.azure.storage.blob.BlobAsyncClient; +import com.azure.storage.blob.BlobContainerAsyncClient; +import com.azure.storage.blob.specialized.BlockBlobAsyncClient; +import org.apache.samza.system.azureblob.compression.Compression; +import org.apache.samza.system.azureblob.compression.CompressionFactory; +import org.apache.samza.system.azureblob.compression.CompressionType; +import org.apache.samza.system.azureblob.producer.AzureBlobWriterMetrics; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.io.EncoderFactory; +import org.apache.avro.specific.SpecificDatumWriter; +import org.apache.avro.specific.SpecificRecord; +import org.apache.samza.SamzaException; +import org.apache.samza.system.OutgoingMessageEnvelope; +import org.apache.samza.system.SystemStream; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Matchers; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + + +@RunWith(PowerMockRunner.class) +@PrepareForTest({BlobContainerAsyncClient.class, BlockBlobAsyncClient.class, AzureBlobAvroWriter.class, AzureBlobOutputStream.class}) +public class TestAzureBlobAvroWriter { + private ThreadPoolExecutor threadPool; + private OutgoingMessageEnvelope ome; + private byte[] encodedRecord; + private AzureBlobAvroWriter azureBlobAvroWriter; + private DataFileWriter mockDataFileWriter; + private AzureBlobOutputStream mockAzureBlobOutputStream; + private BlockBlobAsyncClient mockBlockBlobAsyncClient; + private Compression mockCompression; + + private static final String VALUE = "FAKE_VALUE"; + private static final String SYSTEM_NAME = "FAKE_SYSTEM"; + private static final int THRESHOLD = 100; + + private class SpecificRecordEvent extends org.apache.avro.specific.SpecificRecordBase + implements org.apache.avro.specific.SpecificRecord { + public final org.apache.avro.Schema schema = org.apache.avro.Schema.parse( + "{\"type\":\"record\",\"name\":\"SpecificRecordEvent\",\"namespace\":\"org.apache.samza.events\",\"fields\":[]}"); + + public org.apache.avro.Schema getSchema() { + return schema; + } + + public java.lang.Object get(int field) { + return null; + } + + public void put(int field, Object value) {} + } + + private class GenericRecordEvent implements org.apache.avro.generic.GenericRecord { + public final org.apache.avro.Schema schema = org.apache.avro.Schema.parse( + "{\"type\":\"record\",\"name\":\"GenericRecordEvent\",\"namespace\":\"org.apache.samza.events\",\"fields\":[]}"); + + public org.apache.avro.Schema getSchema() { + return schema; + } + + public java.lang.Object get(String key) { + return null; + } + + public java.lang.Object get(int field) { + return null; + } + + public void put(int field, Object value) {} + public void put(String key, Object value) {} + } + + private OutgoingMessageEnvelope createOME(String streamName) { + SystemStream systemStream = new SystemStream(SYSTEM_NAME, streamName); + SpecificRecord record = new SpecificRecordEvent(); + return new OutgoingMessageEnvelope(systemStream, record); + } + + private OutgoingMessageEnvelope createOMEGenericRecord(String streamName) { + SystemStream systemStream = new SystemStream(SYSTEM_NAME, streamName); + GenericRecord record = new GenericRecordEvent(); + return new OutgoingMessageEnvelope(systemStream, record); + } + + @Before + public void setup() throws Exception { + threadPool = new ThreadPoolExecutor(1, 1, 60, TimeUnit.SECONDS, new LinkedBlockingDeque<>()); + ome = createOME("Topic1"); + + encodedRecord = new byte[100]; + BlobContainerAsyncClient mockContainerAsyncClient = PowerMockito.mock(BlobContainerAsyncClient.class); + mockDataFileWriter = mock(DataFileWriter.class); + mockAzureBlobOutputStream = mock(AzureBlobOutputStream.class); + mockBlockBlobAsyncClient = PowerMockito.mock(BlockBlobAsyncClient.class); + when(mockBlockBlobAsyncClient.getBlobUrl()).thenReturn("https://samza.blob.core.windows.net/fake-blob-url"); + + mockCompression = CompressionFactory.getInstance().getCompression(CompressionType.GZIP); + azureBlobAvroWriter = + spy(new AzureBlobAvroWriter(mockContainerAsyncClient, mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, + 60000, "test", mockDataFileWriter, mockAzureBlobOutputStream, mockBlockBlobAsyncClient, Long.MAX_VALUE, + Long.MAX_VALUE, mockCompression, false)); // keeping blob size and number of records unlimited + doReturn(encodedRecord).when(azureBlobAvroWriter).encodeRecord((IndexedRecord) ome.getMessage()); + } + @Test + public void testWrite() throws Exception { + int numberOfMessages = 10; + for (int i = 0; i < numberOfMessages; ++i) { + azureBlobAvroWriter.write(ome); + } + verify(mockDataFileWriter, times(numberOfMessages)).appendEncoded(ByteBuffer.wrap(encodedRecord)); + } + + @Test + public void testWriteGenericRecord() throws Exception { + OutgoingMessageEnvelope omeGenericRecord = createOMEGenericRecord("Topic1"); + doReturn(encodedRecord).when(azureBlobAvroWriter).encodeRecord((IndexedRecord) omeGenericRecord.getMessage()); + int numberOfMessages = 10; + for (int i = 0; i < numberOfMessages; ++i) { + azureBlobAvroWriter.write(omeGenericRecord); + } + verify(mockDataFileWriter, times(numberOfMessages)).appendEncoded(ByteBuffer.wrap(encodedRecord)); + } + + @Test + public void testWriteByteArray() throws Exception { + OutgoingMessageEnvelope omeEncoded = new OutgoingMessageEnvelope(new SystemStream(SYSTEM_NAME, "Topic1"), "randomString".getBytes()); + int numberOfMessages = 10; + azureBlobAvroWriter.write(ome); + for (int i = 0; i < numberOfMessages; ++i) { + azureBlobAvroWriter.write(omeEncoded); + } + verify(mockDataFileWriter).appendEncoded(ByteBuffer.wrap(encodedRecord)); + verify(mockDataFileWriter, times(numberOfMessages)).appendEncoded(ByteBuffer.wrap((byte[]) omeEncoded.getMessage())); + } + + @Test(expected = IllegalStateException.class) + public void testWriteByteArrayWithoutSchema() throws Exception { + azureBlobAvroWriter = + spy(new AzureBlobAvroWriter(PowerMockito.mock(BlobContainerAsyncClient.class), mock(AzureBlobWriterMetrics.class), + threadPool, THRESHOLD, 60000, "test", + null, null, null, 1000, 100, mockCompression, false)); + OutgoingMessageEnvelope omeEncoded = new OutgoingMessageEnvelope(new SystemStream(SYSTEM_NAME, "Topic1"), new byte[100]); + azureBlobAvroWriter.write(omeEncoded); + } + + @Test(expected = IOException.class) + public void testWriteWhenDataFileWriterFails() throws Exception { + doThrow(new IOException("Failed")).when(mockDataFileWriter).appendEncoded(ByteBuffer.wrap(encodedRecord)); + azureBlobAvroWriter.write(ome); + } + + @Test + public void testClose() throws Exception { + azureBlobAvroWriter.close(); + verify(mockDataFileWriter).close(); + } + + @Test(expected = SamzaException.class) + public void testCloseWhenDataFileWriterFails() throws Exception { + doThrow(new IOException("Failed")).when(mockDataFileWriter).close(); + + azureBlobAvroWriter.flush(); + azureBlobAvroWriter.close(); + verify(mockAzureBlobOutputStream).close(); + } + + @Test(expected = RuntimeException.class) + public void testCloseWhenOutputStreamFails() throws Exception { + doThrow(new IOException("DataFileWriter failed")).when(mockDataFileWriter).close(); + doThrow(new RuntimeException("failed")).when(mockAzureBlobOutputStream).close(); + + azureBlobAvroWriter.close(); + } + + @Test + public void testFlush() throws Exception { + azureBlobAvroWriter.flush(); + verify(mockDataFileWriter).flush(); + } + + @Test(expected = IOException.class) + public void testFlushWhenDataFileWriterFails() throws Exception { + doThrow(new IOException("Failed")).when(mockDataFileWriter).flush(); + azureBlobAvroWriter.flush(); + } + + @Test + public void testMaxBlobSizeExceeded() throws Exception { + String blobUrlPrefix = "test"; + String blobNameRegex = "test/[0-9]{4}/[0-9]{2}/[0-9]{2}/[0-9]{2}/[0-9]{2}-[0-9]{2}-.{8}.avro.gz"; + long maxBlobSize = 1000; + AzureBlobWriterMetrics mockMetrics = mock(AzureBlobWriterMetrics.class); + BlobContainerAsyncClient mockContainerClient = PowerMockito.mock(BlobContainerAsyncClient.class); + azureBlobAvroWriter = spy(new AzureBlobAvroWriter(mockContainerClient, + mockMetrics, threadPool, THRESHOLD, 60000, blobUrlPrefix, + null, null, null, maxBlobSize, 10, mockCompression, true)); + + DataFileWriter mockDataFileWriter1 = mock(DataFileWriter.class); + PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter1); + + BlobAsyncClient mockBlobAsyncClient1 = mock(BlobAsyncClient.class); + doReturn(mockBlobAsyncClient1).when(mockContainerClient).getBlobAsyncClient(Matchers.matches(blobNameRegex)); + BlockBlobAsyncClient mockBlockBlobAsyncClient1 = mock(BlockBlobAsyncClient.class); + doReturn(mockBlockBlobAsyncClient1).when(mockBlobAsyncClient1).getBlockBlobAsyncClient(); + + AzureBlobOutputStream mockAzureBlobOutputStream1 = mock(AzureBlobOutputStream.class); + PowerMockito.whenNew(AzureBlobOutputStream.class).withArguments(mockBlockBlobAsyncClient1, threadPool, + mockMetrics, + (long) 60000, THRESHOLD, mockCompression).thenReturn(mockAzureBlobOutputStream1); + when(mockAzureBlobOutputStream1.getSize()).thenReturn((long) maxBlobSize - 1); + + // first OME creates the first blob + azureBlobAvroWriter.write(ome); + + OutgoingMessageEnvelope ome2 = createOME("Topic2"); + DataFileWriter mockDataFileWriter2 = mock(DataFileWriter.class); + PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter2); + + BlobAsyncClient mockBlobAsyncClient2 = mock(BlobAsyncClient.class); + doReturn(mockBlobAsyncClient2).when(mockContainerClient).getBlobAsyncClient(Matchers.matches(blobNameRegex)); + BlockBlobAsyncClient mockBlockBlobAsyncClient2 = mock(BlockBlobAsyncClient.class); + doReturn(mockBlockBlobAsyncClient2).when(mockBlobAsyncClient2).getBlockBlobAsyncClient(); + + AzureBlobOutputStream mockAzureBlobOutputStream2 = mock(AzureBlobOutputStream.class); + PowerMockito.whenNew(AzureBlobOutputStream.class).withArguments(mockBlockBlobAsyncClient2, threadPool, + mockMetrics, + (long) 60000, THRESHOLD, mockCompression).thenReturn(mockAzureBlobOutputStream2); + when(mockAzureBlobOutputStream2.getSize()).thenReturn((long) maxBlobSize - 1); + + // Second OME creates the second blob because maxBlobSize is 1000 and mockAzureBlobOutputStream.getSize is 999. + azureBlobAvroWriter.write(ome2); + + ArgumentCaptor argument = ArgumentCaptor.forClass(String.class); + verify(mockContainerClient, times(2)).getBlobAsyncClient(argument.capture()); + argument.getAllValues().forEach(blobName -> { + Assert.assertTrue(blobName.contains(blobUrlPrefix)); + }); + List allBlobNames = argument.getAllValues(); + Assert.assertNotEquals(allBlobNames.get(0), allBlobNames.get(1)); + + verify(mockDataFileWriter1).appendEncoded(ByteBuffer.wrap(encodeRecord((IndexedRecord) ome.getMessage()))); + verify(mockDataFileWriter2).appendEncoded(ByteBuffer.wrap(encodeRecord((IndexedRecord) ome2.getMessage()))); + + verify(mockDataFileWriter1).create(((IndexedRecord) ome.getMessage()).getSchema(), mockAzureBlobOutputStream1); + verify(mockDataFileWriter2).create(((IndexedRecord) ome2.getMessage()).getSchema(), mockAzureBlobOutputStream2); + } + + @Test + public void testRecordLimitExceeded() throws Exception { + String blobUrlPrefix = "test"; + String blobNameRegex = "test/[0-9]{4}/[0-9]{2}/[0-9]{2}/[0-9]{2}/[0-9]{2}-[0-9]{2}-.{8}.avro.gz"; + AzureBlobWriterMetrics mockMetrics = mock(AzureBlobWriterMetrics.class); + long maxBlobSize = AzureBlobAvroWriter.DATAFILEWRITER_OVERHEAD + 1000; + long maxRecordsPerBlob = 10; + BlobContainerAsyncClient mockContainerClient = PowerMockito.mock(BlobContainerAsyncClient.class); + azureBlobAvroWriter = spy(new AzureBlobAvroWriter(mockContainerClient, + mockMetrics, threadPool, THRESHOLD, 60000, blobUrlPrefix, + null, null, null, maxBlobSize, maxRecordsPerBlob, mockCompression, true)); + + DataFileWriter mockDataFileWriter1 = mock(DataFileWriter.class); + PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter1); + + BlobAsyncClient mockBlobAsyncClient1 = mock(BlobAsyncClient.class); + doReturn(mockBlobAsyncClient1).when(mockContainerClient).getBlobAsyncClient(Matchers.matches(blobNameRegex)); + BlockBlobAsyncClient mockBlockBlobAsyncClient1 = mock(BlockBlobAsyncClient.class); + doReturn(mockBlockBlobAsyncClient1).when(mockBlobAsyncClient1).getBlockBlobAsyncClient(); + + AzureBlobOutputStream mockAzureBlobOutputStream1 = mock(AzureBlobOutputStream.class); + PowerMockito.whenNew(AzureBlobOutputStream.class).withArguments(mockBlockBlobAsyncClient1, threadPool, + mockMetrics, + (long) 60000, THRESHOLD, mockCompression).thenReturn(mockAzureBlobOutputStream1); + when(mockAzureBlobOutputStream1.getSize()).thenReturn((long) 1); + + // first OME creates the first blob and 11th OME (ome2) creates the second blob. + + for (int i = 0; i < maxRecordsPerBlob; i++) { + azureBlobAvroWriter.write(ome); + } + + OutgoingMessageEnvelope ome2 = createOME("Topic2"); + DataFileWriter mockDataFileWriter2 = mock(DataFileWriter.class); + PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter2); + + BlobAsyncClient mockBlobAsyncClient2 = mock(BlobAsyncClient.class); + doReturn(mockBlobAsyncClient2).when(mockContainerClient).getBlobAsyncClient(Matchers.matches(blobNameRegex)); + BlockBlobAsyncClient mockBlockBlobAsyncClient2 = mock(BlockBlobAsyncClient.class); + doReturn(mockBlockBlobAsyncClient2).when(mockBlobAsyncClient2).getBlockBlobAsyncClient(); + + AzureBlobOutputStream mockAzureBlobOutputStream2 = mock(AzureBlobOutputStream.class); + PowerMockito.whenNew(AzureBlobOutputStream.class).withArguments(mockBlockBlobAsyncClient2, threadPool, + mockMetrics, + (long) 60000, THRESHOLD, mockCompression).thenReturn(mockAzureBlobOutputStream2); + when(mockAzureBlobOutputStream2.getSize()).thenReturn((long) 1); + + azureBlobAvroWriter.write(ome2); + + ArgumentCaptor argument = ArgumentCaptor.forClass(String.class); + verify(mockContainerClient, times(2)).getBlobAsyncClient(argument.capture()); + argument.getAllValues().forEach(blobName -> { + Assert.assertTrue(blobName.contains(blobUrlPrefix)); + }); + List allBlobNames = argument.getAllValues(); + Assert.assertNotEquals(allBlobNames.get(0), allBlobNames.get(1)); + + verify(mockDataFileWriter1, times((int) maxRecordsPerBlob)).appendEncoded(ByteBuffer.wrap(encodeRecord((IndexedRecord) ome.getMessage()))); + verify(mockDataFileWriter2).appendEncoded(ByteBuffer.wrap(encodeRecord((IndexedRecord) ome2.getMessage()))); + + verify(mockDataFileWriter1).create(((IndexedRecord) ome.getMessage()).getSchema(), mockAzureBlobOutputStream1); + verify(mockDataFileWriter2).create(((IndexedRecord) ome2.getMessage()).getSchema(), mockAzureBlobOutputStream2); + } + + @Test + public void testMultipleBlobClose() throws Exception { + String blobUrlPrefix = "test"; + long maxBlobSize = AzureBlobAvroWriter.DATAFILEWRITER_OVERHEAD + 1000; + long maxRecordsPerBlob = 10; + BlobContainerAsyncClient mockContainerClient = PowerMockito.mock(BlobContainerAsyncClient.class); + azureBlobAvroWriter = spy(new AzureBlobAvroWriter(mockContainerClient, + mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, 60000, blobUrlPrefix, + mockDataFileWriter, mockAzureBlobOutputStream, mockBlockBlobAsyncClient, maxBlobSize, maxRecordsPerBlob, + mockCompression, false)); + + DataFileWriter mockDataFileWriter2 = mock(DataFileWriter.class); + AzureBlobOutputStream mockAzureBlobOutputStream2 = mock(AzureBlobOutputStream.class); + + when(mockAzureBlobOutputStream.getSize()).thenReturn((long) 1); + BlobAsyncClient mockBlobAsyncClient = mock(BlobAsyncClient.class); + doReturn(mockBlobAsyncClient).when(mockContainerClient).getBlobAsyncClient(anyString()); + doReturn(mockBlockBlobAsyncClient).when(mockBlobAsyncClient).getBlockBlobAsyncClient(); + PowerMockito.whenNew(AzureBlobOutputStream.class).withAnyArguments().thenReturn(mockAzureBlobOutputStream2); + PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter2); + for (int i = 0; i <= maxRecordsPerBlob; i++) { + azureBlobAvroWriter.write(ome); + } + // first OME creates the first blob and 11th OME creates the second blob. + + azureBlobAvroWriter.close(); + verify(mockDataFileWriter).close(); + verify(mockDataFileWriter2).close(); + } + + @Test + public void testEncodeRecord() throws Exception { + azureBlobAvroWriter = spy(new AzureBlobAvroWriter(PowerMockito.mock(BlobContainerAsyncClient.class), + mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, + 60000, "test", mockDataFileWriter, mockAzureBlobOutputStream, mockBlockBlobAsyncClient, + Long.MAX_VALUE, Long.MAX_VALUE, mockCompression, false)); + IndexedRecord record = new GenericRecordEvent(); + Assert.assertTrue(Arrays.equals(encodeRecord(record), azureBlobAvroWriter.encodeRecord(record))); + } + + @Test + public void testMultipleThreadWrites() throws Exception { + Thread t1 = writeInThread(ome, azureBlobAvroWriter, 10); + OutgoingMessageEnvelope ome2 = createOMEGenericRecord("TOPIC2"); + Thread t2 = writeInThread(ome2, azureBlobAvroWriter, 10); + + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + + verify(mockDataFileWriter, times(10)).appendEncoded(ByteBuffer.wrap(encodedRecord)); + verify(mockDataFileWriter, times(10)).appendEncoded(ByteBuffer.wrap(encodeRecord((IndexedRecord) ome2.getMessage()))); + } + + @Test + public void testMultipleThreadWriteFlush() throws Exception { + Thread t1 = writeInThread(ome, azureBlobAvroWriter, 10); + Thread t2 = flushInThread(azureBlobAvroWriter); + + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + + verify(mockDataFileWriter, times(10)).appendEncoded(ByteBuffer.wrap(encodedRecord)); + verify(mockDataFileWriter).flush(); + } + + @Test + public void testMultipleThreadWriteFlushInBoth() throws Exception { + Thread t1 = writeFlushInThread(ome, azureBlobAvroWriter, 10); + OutgoingMessageEnvelope ome2 = createOMEGenericRecord("TOPIC2"); + Thread t2 = writeFlushInThread(ome2, azureBlobAvroWriter, 10); + + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + + verify(mockDataFileWriter, times(10)).appendEncoded(ByteBuffer.wrap(encodedRecord)); + verify(mockDataFileWriter, times(10)).appendEncoded(ByteBuffer.wrap(encodeRecord((IndexedRecord) ome2.getMessage()))); + verify(mockDataFileWriter, times(2)).flush(); + } + + @Test + public void testMultipleThreadWriteFlushFinallyClose() throws Exception { + Thread t1 = writeFlushInThread(ome, azureBlobAvroWriter, 10); + OutgoingMessageEnvelope ome2 = createOMEGenericRecord("TOPIC2"); + Thread t2 = writeFlushInThread(ome2, azureBlobAvroWriter, 10); + + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + azureBlobAvroWriter.close(); + + verify(mockDataFileWriter, times(10)).appendEncoded(ByteBuffer.wrap(encodedRecord)); + verify(mockDataFileWriter, times(10)).appendEncoded(ByteBuffer.wrap(encodeRecord((IndexedRecord) ome2.getMessage()))); + verify(mockDataFileWriter, times(2)).flush(); + verify(mockDataFileWriter).close(); + } + + private byte[] encodeRecord(IndexedRecord record) throws Exception { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + Schema schema = record.getSchema(); + EncoderFactory encoderfactory = new EncoderFactory(); + BinaryEncoder encoder = encoderfactory.binaryEncoder(out, null); + DatumWriter writer; + if (record instanceof SpecificRecord) { + writer = new SpecificDatumWriter<>(schema); + } else { + writer = new GenericDatumWriter<>(schema); + } + writer.write(record, encoder); + encoder.flush(); //encoder may buffer + return out.toByteArray(); + } + + private Thread writeInThread(OutgoingMessageEnvelope ome, AzureBlobAvroWriter azureBlobAvroWriter, + int numberOfSends) { + Thread t = new Thread() { + @Override + public void run() { + try { + for (int i = 0; i < numberOfSends; i++) { + azureBlobAvroWriter.write(ome); + } + } catch (IOException e) { + throw new SamzaException(e); + } + } + }; + return t; + } + + private Thread flushInThread(AzureBlobAvroWriter azureBlobAvroWriter) { + Thread t = new Thread() { + @Override + public void run() { + try { + azureBlobAvroWriter.flush(); + } catch (IOException e) { + throw new SamzaException(e); + } + } + }; + return t; + } + + private Thread writeFlushInThread(OutgoingMessageEnvelope ome, AzureBlobAvroWriter azureBlobAvroWriter, + int numberOfSends) { + Thread t = new Thread() { + @Override + public void run() { + try { + for (int i = 0; i < numberOfSends; i++) { + azureBlobAvroWriter.write(ome); + } + azureBlobAvroWriter.flush(); + } catch (IOException e) { + throw new SamzaException(e); + } + } + }; + return t; + } +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java new file mode 100644 index 0000000000..fa21934e29 --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java @@ -0,0 +1,346 @@ +/* + * 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.system.azureblob.avro; + +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.implementation.util.FluxUtil; +import java.util.Arrays; +import org.apache.samza.system.azureblob.compression.Compression; +import org.apache.samza.system.azureblob.producer.AzureBlobWriterMetrics; +import com.azure.storage.blob.specialized.BlockBlobAsyncClient; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Base64; +import java.util.Map; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.ArgumentCaptor; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyList; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({BlockBlobAsyncClient.class}) +public class TestAzureBlobOutputStream { + private ThreadPoolExecutor threadPool; + private ByteArrayOutputStream mockByteArrayOutputStream; + private static final int THRESHOLD = 100; + private BlockBlobAsyncClient mockBlobAsyncClient; + private AzureBlobOutputStream azureBlobOutputStream; + private static final String RANDOM_STRING = "roZzozzLiR7GCEjcB0UsRUNgBAip8cSLGXQSo3RQvbIDoxOaaRs4hrec2s5rMPWgTPRY4UnE959worEtyhRjwUFnRnVuNFZ554yuPQCbI69qFkQX7MmrB4blmpSnFeGjWKjFjIRLFNVSsQBYMkr5jT4T83uVtuGumsjACVrpcilihdd194H8Y71rQcrXZoTQtw5OvmPicbwptawpHoRNzHihyaDVYgAs0dQbvVEu1gitKpamzYdMLFtc5h8PFZSVEB"; + private static final byte[] BYTES = RANDOM_STRING.substring(0, THRESHOLD).getBytes(); + private static final byte[] COMPRESSED_BYTES = RANDOM_STRING.substring(0, THRESHOLD / 2).getBytes(); + private AzureBlobWriterMetrics mockMetrics; + private Compression mockCompression; + + @Before + public void setup() throws Exception { + threadPool = new ThreadPoolExecutor(1, 1, 60, + TimeUnit.SECONDS, new LinkedBlockingDeque()); + + + mockByteArrayOutputStream = spy(new ByteArrayOutputStream(THRESHOLD)); + + mockBlobAsyncClient = PowerMockito.mock(BlockBlobAsyncClient.class); + when(mockBlobAsyncClient.stageBlock(anyString(), any(), anyLong())).thenReturn( + Mono.just(new SimpleResponse(null, 200, null, null)).flatMap(FluxUtil::toMono)); + when(mockBlobAsyncClient.commitBlockListWithResponse(any(), any(), any(), any(), any())).thenReturn( + Mono.just(new SimpleResponse(null, 200, null, null))); + + when(mockBlobAsyncClient.getBlobUrl()).thenReturn("https://samza.blob.core.windows.net/fake-blob-url"); + + mockMetrics = mock(AzureBlobWriterMetrics.class); + + mockCompression = mock(Compression.class); + doReturn(COMPRESSED_BYTES).when(mockCompression).compress(BYTES); + + azureBlobOutputStream = spy(new AzureBlobOutputStream(mockBlobAsyncClient, threadPool, mockMetrics, + 60000, THRESHOLD, mockByteArrayOutputStream, mockCompression)); + } + + @Test + public void testWrite() { + byte[] b = new byte[THRESHOLD - 10]; + azureBlobOutputStream.write(b, 0, THRESHOLD - 10); + verify(mockBlobAsyncClient, never()).stageBlock(any(), any(), anyLong()); // since size of byte[] written is less than threshold + verify(mockMetrics).updateWriteByteMetrics(THRESHOLD - 10); + verify(mockMetrics, never()).updateAzureUploadMetrics(); + } + + @Test + public void testWriteLargerThanThreshold() { + byte[] largeRecord = RANDOM_STRING.substring(0, 2 * THRESHOLD).getBytes(); + byte[] largeRecordFirstHalf = RANDOM_STRING.substring(0, THRESHOLD).getBytes(); + byte[] largeRecordSecondHalf = RANDOM_STRING.substring(THRESHOLD, 2 * THRESHOLD).getBytes(); + + byte[] compressB1 = RANDOM_STRING.substring(0, THRESHOLD / 2).getBytes(); + byte[] compressB2 = RANDOM_STRING.substring(THRESHOLD / 2, THRESHOLD).getBytes(); + + doReturn(compressB1).when(mockCompression).compress(largeRecordFirstHalf); + doReturn(compressB2).when(mockCompression).compress(largeRecordSecondHalf); + + azureBlobOutputStream.write(largeRecord, 0, 2 * THRESHOLD); + // azureBlobOutputStream.close waits on the CompletableFuture which does the actual stageBlock in uploadBlockAsync + azureBlobOutputStream.close(); + + // invoked 2 times for the data which is 2*threshold + verify(mockCompression).compress(largeRecordFirstHalf); + verify(mockCompression).compress(largeRecordSecondHalf); + ArgumentCaptor argument0 = ArgumentCaptor.forClass(Flux.class); + ArgumentCaptor argument1 = ArgumentCaptor.forClass(Flux.class); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(0)), argument0.capture(), eq((long) compressB1.length)); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(1)), argument1.capture(), eq((long) compressB2.length)); + Assert.assertEquals(ByteBuffer.wrap(compressB1), argument0.getAllValues().get(0).blockFirst()); + Assert.assertEquals(ByteBuffer.wrap(compressB2), argument1.getAllValues().get(0).blockFirst()); + verify(mockMetrics).updateWriteByteMetrics(2 * THRESHOLD); + verify(mockMetrics, times(2)).updateAzureUploadMetrics(); + } + + @Test + public void testWriteLargeRecordWithSmallRecordInBuffer() { + byte[] halfBlock = new byte[THRESHOLD / 2]; + byte[] fullBlock = new byte[THRESHOLD]; + byte[] largeRecord = new byte[2 * THRESHOLD]; + byte[] fullBlockCompressedByte = new byte[50]; + byte[] halfBlockCompressedByte = new byte[25]; + doReturn(fullBlockCompressedByte).when(mockCompression).compress(fullBlock); + doReturn(halfBlockCompressedByte).when(mockCompression).compress(halfBlock); + + // FIRST write a small record = same as half block + azureBlobOutputStream.write(halfBlock, 0, THRESHOLD / 2); + verify(mockMetrics).updateWriteByteMetrics(THRESHOLD / 2); + + // SECOND write the large record + azureBlobOutputStream.write(largeRecord, 0, 2 * THRESHOLD); + verify(mockMetrics).updateWriteByteMetrics(2 * THRESHOLD); + + azureBlobOutputStream.flush(); // to flush out buffered data + + // azureBlobOutputStream.close waits on the CompletableFuture which does the actual stageBlock in uploadBlockAsync + azureBlobOutputStream.close(); + + verify(mockCompression, times(2)).compress(fullBlock); + verify(mockCompression).compress(halfBlock); + + ArgumentCaptor argument = ArgumentCaptor.forClass(Flux.class); + ArgumentCaptor argument2 = ArgumentCaptor.forClass(Flux.class); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(0)), argument.capture(), eq((long) fullBlockCompressedByte.length)); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(1)), argument.capture(), eq((long) fullBlockCompressedByte.length)); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(2)), argument2.capture(), eq((long) halfBlockCompressedByte.length)); + argument.getAllValues().forEach(flux -> { + Assert.assertEquals(ByteBuffer.wrap(fullBlockCompressedByte), flux.blockFirst()); + }); + Assert.assertEquals(ByteBuffer.wrap(halfBlockCompressedByte), ((Flux) argument2.getValue()).blockFirst()); + verify(mockMetrics, times(3)).updateAzureUploadMetrics(); + } + + + @Test + public void testWriteThresholdCrossed() throws Exception { + azureBlobOutputStream.write(BYTES, 0, THRESHOLD / 2); + azureBlobOutputStream.write(BYTES, THRESHOLD / 2, THRESHOLD / 2); + // azureBlobOutputStream.close waits on the CompletableFuture which does the actual stageBlock in uploadBlockAsync + azureBlobOutputStream.close(); + + verify(mockCompression).compress(BYTES); + ArgumentCaptor argument = ArgumentCaptor.forClass(Flux.class); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(0)), argument.capture(), eq((long) COMPRESSED_BYTES.length)); // since size of byte[] written is less than threshold + Assert.assertEquals(ByteBuffer.wrap(COMPRESSED_BYTES), ((Flux) argument.getValue()).blockFirst()); + verify(mockMetrics, times(2)).updateWriteByteMetrics(THRESHOLD / 2); + verify(mockMetrics, times(1)).updateAzureUploadMetrics(); + } + + @Test(expected = RuntimeException.class) + public void testWriteFailed() { + when(mockBlobAsyncClient.stageBlock(anyString(), any(), anyLong())).thenThrow(new Exception("Test Failed")); + + byte[] b = new byte[100]; + azureBlobOutputStream.write(b, 0, THRESHOLD); // threshold crossed so stageBlock is scheduled. + // azureBlobOutputStream.close waits on the CompletableFuture which does the actual stageBlock in uploadBlockAsync + azureBlobOutputStream.close(); + } + + @Test + public void testClose() { + azureBlobOutputStream.write(BYTES, 0, THRESHOLD); + int blockNum = 0; + String blockId = String.format("%05d", blockNum); + String blockIdEncoded = Base64.getEncoder().encodeToString(blockId.getBytes()); + + PowerMockito.doAnswer(invocation -> { + ArrayList blockListArg = (ArrayList) invocation.getArguments()[0]; + String blockIdArg = (String) blockListArg.toArray()[0]; + Assert.assertEquals(blockIdEncoded, blockIdArg); + Map blobMetadata = (Map) invocation.getArguments()[2]; + Assert.assertEquals(blobMetadata.get(AzureBlobOutputStream.BLOB_RAW_SIZE_BYTES_METADATA), Long.toString(THRESHOLD)); + return Mono.just(new SimpleResponse(null, 200, null, null)); + }).when(mockBlobAsyncClient).commitBlockListWithResponse(anyList(), any(), any(), any(), any()); + + azureBlobOutputStream.close(); + verify(mockMetrics).updateAzureCommitMetrics(); + } + + @Test + public void testCloseMultipleBlocks() { + azureBlobOutputStream.write(BYTES, 0, THRESHOLD); + azureBlobOutputStream.write(BYTES, 0, THRESHOLD); + + int blockNum = 0; + String blockId = String.format("%05d", blockNum); + String blockIdEncoded = Base64.getEncoder().encodeToString(blockId.getBytes()); + + int blockNum1 = 1; + String blockId1 = String.format("%05d", blockNum1); + String blockIdEncoded1 = Base64.getEncoder().encodeToString(blockId1.getBytes()); + + PowerMockito.doAnswer(invocation -> { + ArrayList blockListArg = (ArrayList) invocation.getArguments()[0]; + Assert.assertEquals(Arrays.asList(blockIdEncoded, blockIdEncoded1), blockListArg); + Map blobMetadata = (Map) invocation.getArguments()[2]; + Assert.assertEquals(blobMetadata.get(AzureBlobOutputStream.BLOB_RAW_SIZE_BYTES_METADATA), Long.toString(2 * THRESHOLD)); + return Mono.just(new SimpleResponse(null, 200, null, null)); + }).when(mockBlobAsyncClient).commitBlockListWithResponse(anyList(), any(), any(), any(), any()); + + azureBlobOutputStream.close(); + verify(mockMetrics).updateAzureCommitMetrics(); + } + + @Test(expected = RuntimeException.class) + public void testCloseFailed() { + when(mockBlobAsyncClient.commitBlockListWithResponse(anyList(), any(), any(), any(), any())) + .thenReturn(Mono.error(new Exception("Test Failed"))); + + byte[] b = new byte[100]; + azureBlobOutputStream.write(b, 0, THRESHOLD); + azureBlobOutputStream.close(); + } + + @Test + public void testMultipleClose() { + azureBlobOutputStream.write(BYTES, 0, THRESHOLD); + azureBlobOutputStream.close(); + azureBlobOutputStream.close(); + } + + @Test + public void testFlush() throws Exception { + azureBlobOutputStream.write(BYTES); + azureBlobOutputStream.flush(); + // azureBlobOutputStream.close waits on the CompletableFuture which does the actual stageBlock in uploadBlockAsync + azureBlobOutputStream.close(); + + int blockNum = 0; // as there is only one block and its id will be 0 + String blockId = String.format("%05d", blockNum); + String blockIdEncoded = Base64.getEncoder().encodeToString(blockId.getBytes()); + + verify(mockCompression).compress(BYTES); + ArgumentCaptor argument = ArgumentCaptor.forClass(Flux.class); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded), argument.capture(), eq((long) COMPRESSED_BYTES.length)); // since size of byte[] written is less than threshold + Assert.assertEquals(ByteBuffer.wrap(COMPRESSED_BYTES), ((Flux) argument.getValue()).blockFirst()); + verify(mockMetrics).updateAzureUploadMetrics(); + } + + @Test (expected = RuntimeException.class) + public void testFlushFailed() throws IOException { + azureBlobOutputStream.write(BYTES); + when(mockBlobAsyncClient.stageBlock(anyString(), any(), anyLong())) + .thenReturn(Mono.error(new Exception("Test Failed"))); + + azureBlobOutputStream.flush(); + // azureBlobOutputStream.close waits on the CompletableFuture which does the actual stageBlock in uploadBlockAsync + azureBlobOutputStream.close(); + verify(mockMetrics).updateAzureUploadMetrics(); + } + + + @Test + public void testReleaseBuffer() throws Exception { + azureBlobOutputStream.releaseBuffer(); + verify(mockByteArrayOutputStream).close(); + } + + @Test(expected = IllegalStateException.class) + public void testWriteAfterReleaseBuffer() throws Exception { + azureBlobOutputStream.releaseBuffer(); + azureBlobOutputStream.write(new byte[10], 0, 10); + } + + @Test + public void testCloseAfterReleaseBuffer() throws Exception { + azureBlobOutputStream.write(BYTES, 0, 100); + azureBlobOutputStream.releaseBuffer(); + azureBlobOutputStream.close(); + // mockByteArrayOutputStream.close called only once during releaseBuffer and not during azureBlobOutputStream.close + verify(mockByteArrayOutputStream).close(); + // azureBlobOutputStream.close still commits the list of blocks. + verify(mockBlobAsyncClient).commitBlockListWithResponse(any(), any(), any(), any(), any()); + } + + @Test + public void testFlushAfterReleaseBuffer() throws Exception { + azureBlobOutputStream.releaseBuffer(); + azureBlobOutputStream.flush(); // becomes no-op after release buffer + verify(mockBlobAsyncClient, never()).stageBlock(anyString(), any(), anyLong()); + } + + @Test + public void testGetSize() throws Exception { + Assert.assertEquals(0, azureBlobOutputStream.getSize()); + azureBlobOutputStream.write(BYTES, 0, BYTES.length); + Assert.assertEquals(BYTES.length, azureBlobOutputStream.getSize()); + } + + @Test + public void testGetSizeAfterFlush() throws Exception { + azureBlobOutputStream.write(BYTES, 0, BYTES.length); + Assert.assertEquals(BYTES.length, azureBlobOutputStream.getSize()); + azureBlobOutputStream.flush(); + Assert.assertEquals(BYTES.length, azureBlobOutputStream.getSize()); + azureBlobOutputStream.write(BYTES, 0, BYTES.length - 10); + Assert.assertEquals(BYTES.length + BYTES.length - 10, azureBlobOutputStream.getSize()); + } + + private String blockIdEncoded(int blockNum) { + String blockId = String.format("%05d", blockNum); + return Base64.getEncoder().encodeToString(blockId.getBytes()); + } +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/azureblob/compression/TestGzipCompression.java b/samza-azure/src/test/java/org/apache/samza/system/azureblob/compression/TestGzipCompression.java new file mode 100644 index 0000000000..e0cfc5320b --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/azureblob/compression/TestGzipCompression.java @@ -0,0 +1,76 @@ +/* + * 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.system.azureblob.compression; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.zip.GZIPOutputStream; +import org.junit.Before; +import org.junit.Test; +import org.junit.Assert; + +public class TestGzipCompression { + private GzipCompression gzipCompression; + + @Before + public void setup() { + gzipCompression = new GzipCompression(); + } + + @Test + public void testCompression() throws IOException { + byte[] input = "This is fake input data".getBytes(); + byte[] result = compress(input); + + Assert.assertArrayEquals(gzipCompression.compress(input), result); + } + + @Test + public void testCompressionEmpty() throws IOException { + byte[] input = "".getBytes(); + byte[] result = compress(input); + + Assert.assertArrayEquals(gzipCompression.compress(input), result); + } + + @Test(expected = RuntimeException.class) + public void testCompressionNull() { + byte[] input = null; + gzipCompression.compress(input); + } + + @Test + public void testCompressionZero() throws IOException { + byte[] input = new byte[100]; + byte[] result = compress(input); + + Assert.assertArrayEquals(gzipCompression.compress(input), result); + } + + private byte[] compress(byte[] input) throws IOException { + ByteArrayOutputStream bos = new ByteArrayOutputStream(input.length); + GZIPOutputStream gzipOS = new GZIPOutputStream(bos); + gzipOS.write(input); + gzipOS.close(); + gzipOS.close(); + bos.close(); + return bos.toByteArray(); + } +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/azureblob/producer/TestAzureBlobSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/azureblob/producer/TestAzureBlobSystemProducer.java new file mode 100644 index 0000000000..2969cd02e5 --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/azureblob/producer/TestAzureBlobSystemProducer.java @@ -0,0 +1,594 @@ +/* + * 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.system.azureblob.producer; + +import org.apache.samza.system.azureblob.AzureBlobBasicMetrics; +import org.apache.samza.system.azureblob.AzureBlobConfig; +import org.apache.samza.system.azureblob.avro.AzureBlobAvroWriter; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.system.OutgoingMessageEnvelope; +import org.apache.samza.system.SystemProducerException; +import org.apache.samza.system.SystemStream; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({AzureBlobSystemProducer.class, ThreadPoolExecutor.class}) +public class TestAzureBlobSystemProducer { + + private final static String SYSTEM_NAME = "FAKE_SYSTEM"; + private final static String SOURCE = "FAKE_SOURCE"; + private final static String STREAM = "FAKE_STREAM"; + private final static String ACCOUNT_NAME = "FAKE_ACCOUNT_NAME"; + private final static String ACCOUNT_KEY = "FAKE_ACCOUNT_KEY"; + + private OutgoingMessageEnvelope ome; + private AzureBlobSystemProducer systemProducer; + private MetricsRegistry mockMetricsRegistry; + private AzureBlobWriter mockAzureWriter; + private ThreadPoolExecutor mockThreadPoolExecutor; + private Counter mockErrorCounter; + private boolean exceptionOccured = false; + + @Before + public void setup() throws Exception { + + AzureBlobConfig azureBlobConfig = new AzureBlobConfig(getBasicConfigs()); + + mockMetricsRegistry = mock(MetricsRegistry.class); + mockErrorCounter = mock(Counter.class); + + when(mockMetricsRegistry.newCounter(anyString(), anyString())).thenReturn(mock(Counter.class)); + when(mockMetricsRegistry.newCounter(SOURCE, AzureBlobBasicMetrics.EVENT_PRODUCE_ERROR)).thenReturn(mockErrorCounter); + + ome = createOME(STREAM); + + mockThreadPoolExecutor = spy(new ThreadPoolExecutor(1, 1, 60, TimeUnit.SECONDS, + new LinkedBlockingDeque())); + + PowerMockito.whenNew(ThreadPoolExecutor.class).withAnyArguments().thenReturn(mockThreadPoolExecutor); + + mockAzureWriter = mock(AzureBlobWriter.class); + doNothing().when(mockAzureWriter).close(); + + systemProducer = spy(new AzureBlobSystemProducer(SYSTEM_NAME, azureBlobConfig, mockMetricsRegistry)); + // use mock writer impl + doReturn(mockAzureWriter).when(systemProducer).createNewWriter(anyString(), any()); + // bypass Azure connection setup + doNothing().when(systemProducer).setupAzureContainer(anyString(), anyString()); + } + + @Test + public void testStart() { + + systemProducer.start(); + verify(systemProducer).setupAzureContainer(ACCOUNT_NAME, ACCOUNT_KEY); + } + + public void testMultipleStart() { + systemProducer.start(); + systemProducer.start(); + } + + @Test + public void testStop() throws Exception { + doNothing().when(mockAzureWriter).close(); + + systemProducer.register(SOURCE); + systemProducer.start(); + systemProducer.send(SOURCE, ome); + systemProducer.flush(SOURCE); + systemProducer.stop(); + + verify(mockAzureWriter).flush(); // called during flush IN STOP + verify(mockAzureWriter).close(); // called during flush in STOP + } + + @Test + public void testStopBeforeFlush() throws Exception { + + systemProducer.register(SOURCE); + systemProducer.start(); + systemProducer.send(SOURCE, ome); + systemProducer.stop(); + + verify(mockAzureWriter).flush(); // called during flush IN STOP + verify(mockAzureWriter).close(); // called during flush in STOP + } + + @Test(expected = SystemProducerException.class) + public void testStopWhenThreadpoolShutdownFails() throws Exception { + doThrow(new SecurityException("failed")).when(mockThreadPoolExecutor).shutdown(); + systemProducer.start(); + systemProducer.stop(); + } + + @Test (expected = SystemProducerException.class) + public void testStopWhenWriterFails() throws IOException { + doThrow(new SystemProducerException("Failed")).when(mockAzureWriter).flush(); + systemProducer.register(SOURCE); + systemProducer.start(); + systemProducer.send(SOURCE, ome); + systemProducer.stop(); + } + + @Test(expected = SystemProducerException.class) + public void testRegisterAfterStart() throws Exception { + systemProducer.start(); + systemProducer.register(SOURCE); + } + + @Test + public void testRegisterMetrics() throws Exception { + systemProducer.register(SOURCE); + + // verify that new counter for system was created during constructor of producer + verify(mockMetricsRegistry).newCounter( + String.format(AzureBlobSystemProducerMetrics.SYSTEM_METRIC_FORMAT, ACCOUNT_NAME, SYSTEM_NAME), + AzureBlobBasicMetrics.EVENT_WRITE_RATE); + // verify that new counter for source was created during register + verify(mockMetricsRegistry).newCounter(SOURCE, AzureBlobBasicMetrics.EVENT_WRITE_RATE); + verify(mockMetricsRegistry).newCounter(SOURCE, AzureBlobBasicMetrics.EVENT_WRITE_BYTE_RATE); + verify(mockMetricsRegistry).newCounter(SOURCE, AzureBlobBasicMetrics.EVENT_PRODUCE_ERROR); + } + + @Test + public void testRegisterWithSystemName() throws Exception { + systemProducer.register(SYSTEM_NAME); + + // verify that new counter for system was created during constructor of producer but not during register + verify(mockMetricsRegistry).newCounter( + String.format(AzureBlobSystemProducerMetrics.SYSTEM_METRIC_FORMAT, ACCOUNT_NAME, SYSTEM_NAME), + AzureBlobBasicMetrics.EVENT_WRITE_RATE); + } + + @Test + public void testFlush() throws IOException { + + systemProducer.register(SOURCE); + systemProducer.start(); + systemProducer.send(SOURCE, ome); + systemProducer.flush(SOURCE); + + verify(mockAzureWriter).flush(); // called during flush + verify(mockAzureWriter).close(); // called during flush + } + + @Test(expected = SystemProducerException.class) + public void testFlushBeforeStart() throws Exception { + systemProducer.flush(SOURCE); + } + + @Test(expected = SystemProducerException.class) + public void testFlushBeforeRegister() throws Exception { + systemProducer.start(); + systemProducer.flush(SOURCE); + } + + @Test + public void testFlushWhenWriterUploadFails() throws Exception { + doThrow(new SystemProducerException("failed")).when(mockAzureWriter).flush(); + + systemProducer.register(SOURCE); + systemProducer.start(); + systemProducer.send(SOURCE, ome); + try { + systemProducer.flush(SOURCE); + Assert.fail("Expected exception not thrown."); + } catch (SystemProducerException e) { + } + + verify(mockErrorCounter).inc(); + } + + @Test + public void testFlushWhenWriterCloseFails() throws Exception { + doThrow(new SystemProducerException("failed")).when(mockAzureWriter).close(); + + systemProducer.register(SOURCE); + systemProducer.start(); + systemProducer.send(SOURCE, ome); + try { + systemProducer.flush(SOURCE); + Assert.fail("Expected exception not thrown."); + } catch (SystemProducerException e) { + } + verify(mockErrorCounter).inc(); + } + + @Test + public void testSend() throws IOException { + int numberOfMessages = 10; + Counter mockWriteCounter = mock(Counter.class); + when(mockMetricsRegistry.newCounter(SOURCE, AzureBlobBasicMetrics.EVENT_WRITE_RATE)).thenReturn(mockWriteCounter); + + systemProducer.register(SOURCE); + systemProducer.start(); + for (int i = 0; i < numberOfMessages; i++) { + systemProducer.send(SOURCE, ome); + } + verify(mockAzureWriter, times(numberOfMessages)).write(ome); + + // verify metrics + verify(mockWriteCounter, times(numberOfMessages)).inc(); + } + + @Test + public void testSendWhenWriterCreateFails() throws Exception { + AzureBlobConfig azureBlobConfig = new AzureBlobConfig(getBasicConfigs()); + + AzureBlobSystemProducer systemProducer = spy(new AzureBlobSystemProducer(SYSTEM_NAME, azureBlobConfig, + mockMetricsRegistry)); + PowerMockito.whenNew(AzureBlobAvroWriter.class).withAnyArguments().thenThrow(new SystemProducerException("Failed")); + // bypass Azure connection setup + doNothing().when(systemProducer).setupAzureContainer(anyString(), anyString()); + + systemProducer.register(SOURCE); + systemProducer.start(); + try { + systemProducer.send(SOURCE, ome); + Assert.fail("Expected exception not thrown."); + } catch (SystemProducerException e) { + } + verify(mockErrorCounter).inc(); + } + + @Test + public void testSendWhenWriterFails() throws Exception { + + doThrow(new SystemProducerException("failed")).when(mockAzureWriter).write(ome); + + systemProducer.register(SOURCE); + systemProducer.start(); + try { + systemProducer.send(SOURCE, ome); + Assert.fail("Expected exception not thrown."); + } catch (SystemProducerException e) { + } + verify(mockErrorCounter).inc(); + } + + @Test + public void testMutipleThread() throws Exception { + String source1 = "FAKE_SOURCE_1"; + String source2 = "FAKE_SOURCE_2"; + + String stream1 = "FAKE_STREAM_1"; + String stream2 = "FAKE_STREAM_2"; + + int sendsInFirstThread = 10; + int sendsInSecondThread = 20; + + OutgoingMessageEnvelope ome1 = createOME(stream1); + OutgoingMessageEnvelope ome2 = createAnotherOME(stream2); + + AzureBlobWriter mockAzureWriter1 = mock(AzureBlobWriter.class); + doNothing().when(mockAzureWriter1).close(); + + AzureBlobWriter mockAzureWriter2 = mock(AzureBlobWriter.class); + doNothing().when(mockAzureWriter2).close(); + + AzureBlobConfig azureBlobConfig = new AzureBlobConfig(getBasicConfigs()); + + AzureBlobSystemProducer systemProducer = spy(new AzureBlobSystemProducer(SYSTEM_NAME, azureBlobConfig, mockMetricsRegistry)); + // bypass Azure connection setup + doNothing().when(systemProducer).setupAzureContainer(anyString(), anyString()); + + doReturn(mockAzureWriter1).when(systemProducer).getOrCreateWriter(source1, ome1); + doReturn(mockAzureWriter2).when(systemProducer).getOrCreateWriter(source2, ome2); + + systemProducer.register(source1); + systemProducer.register(source2); + systemProducer.start(); + Thread t1 = sendFlushInThread(source1, ome1, systemProducer, sendsInFirstThread); + Thread t2 = sendFlushInThread(source2, ome2, systemProducer, sendsInSecondThread); + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + systemProducer.stop(); + verify(mockAzureWriter1, times(sendsInFirstThread)).write(ome1); + verify(mockAzureWriter2, times(sendsInSecondThread)).write(ome2); + } + + @Test + public void testMutipleThreadOneWriterFails() throws Exception { + String source1 = "FAKE_SOURCE_1"; + String source2 = "FAKE_SOURCE_2"; + + String stream1 = "FAKE_STREAM_1"; + String stream2 = "FAKE_STREAM_2"; + + int sendsInFirstThread = 10; + int sendsInSecondThread = 20; + + OutgoingMessageEnvelope ome1 = createOME(stream1); + OutgoingMessageEnvelope ome2 = createAnotherOME(stream2); + + AzureBlobWriter mockAzureWriter1 = mock(AzureBlobWriter.class); + doThrow(new SystemProducerException("failed")).when(mockAzureWriter1).write(ome1); + doNothing().when(mockAzureWriter1).close(); + + AzureBlobWriter mockAzureWriter2 = mock(AzureBlobWriter.class); + doNothing().when(mockAzureWriter2).close(); + + AzureBlobConfig azureBlobConfig = new AzureBlobConfig(getBasicConfigs()); + + AzureBlobSystemProducer systemProducer = spy(new AzureBlobSystemProducer(SYSTEM_NAME, azureBlobConfig, mockMetricsRegistry)); + // bypass Azure connection setup + doNothing().when(systemProducer).setupAzureContainer(anyString(), anyString()); + + doReturn(mockAzureWriter1).when(systemProducer).getOrCreateWriter(source1, ome1); + doReturn(mockAzureWriter2).when(systemProducer).getOrCreateWriter(source2, ome2); + + + systemProducer.register(source1); + systemProducer.register(source2); + systemProducer.start(); + Thread t1 = sendFlushInThread(source1, ome1, systemProducer, sendsInFirstThread); + Thread t2 = sendFlushInThread(source2, ome2, systemProducer, sendsInSecondThread); + + Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler() { + public void uncaughtException(Thread th, Throwable ex) { + if (ex instanceof SystemProducerException) { + exceptionOccured = true; + } + } + }; + t1.setUncaughtExceptionHandler(handler); + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + + systemProducer.stop(); + + if (!exceptionOccured) { + Assert.fail("Expected SystemProducerException but did not occur."); + } + verify(mockAzureWriter1).write(ome1); + verify(mockAzureWriter2, times(sendsInSecondThread)).write(ome2); + } + + @Test + public void testMutipleThreadSendFlushToSingleWriter() throws Exception { + String source1 = "FAKE_SOURCE_1"; + + String stream1 = "FAKE_STREAM_1"; + + int sendsInFirstThread = 10; + int sendsInSecondThread = 20; + + OutgoingMessageEnvelope ome1 = createOME(stream1); + + AzureBlobWriter mockAzureWriter1 = mock(AzureBlobWriter.class); + doNothing().when(mockAzureWriter1).close(); + + AzureBlobConfig azureBlobConfig = new AzureBlobConfig(getBasicConfigs()); + + AzureBlobSystemProducer systemProducer = spy(new AzureBlobSystemProducer(SYSTEM_NAME, azureBlobConfig, mockMetricsRegistry)); + // bypass Azure connection setup + doNothing().when(systemProducer).setupAzureContainer(anyString(), anyString()); + + systemProducer.register(source1); + systemProducer.start(); + + doReturn(mockAzureWriter1).when(systemProducer).createNewWriter(anyString(), any()); + + Thread t1 = sendFlushInThread(source1, ome1, systemProducer, sendsInFirstThread); + Thread t2 = sendFlushInThread(source1, ome1, systemProducer, sendsInSecondThread); + + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + systemProducer.stop(); + verify(mockAzureWriter1, times(sendsInFirstThread + sendsInSecondThread)).write(ome1); + verify(mockAzureWriter1, times(2)).flush(); + verify(mockAzureWriter1, times(2)).close(); + } + + @Test + public void testMutipleThreadSendToSingleWriter() throws Exception { + String source1 = "FAKE_SOURCE_1"; + + String stream1 = "FAKE_STREAM_1"; + + int sendsInFirstThread = 10; + int sendsInSecondThread = 20; + + OutgoingMessageEnvelope ome1 = createOME(stream1); + OutgoingMessageEnvelope ome2 = createAnotherOME(stream1); + + AzureBlobWriter mockAzureWriter1 = mock(AzureBlobWriter.class); + doNothing().when(mockAzureWriter1).close(); + + AzureBlobConfig azureBlobConfig = new AzureBlobConfig(getBasicConfigs()); + + AzureBlobSystemProducer systemProducer = spy(new AzureBlobSystemProducer(SYSTEM_NAME, azureBlobConfig, mockMetricsRegistry)); + // bypass Azure connection setup + doNothing().when(systemProducer).setupAzureContainer(anyString(), anyString()); + + doReturn(mockAzureWriter1).when(systemProducer).createNewWriter(anyString(), any()); + + systemProducer.register(source1); + systemProducer.start(); + Thread t1 = new Thread() { + @Override + public void run() { + for (int i = 0; i < sendsInFirstThread; i++) { + systemProducer.send(source1, ome1); + } + } + }; + Thread t2 = new Thread() { + @Override + public void run() { + for (int i = 0; i < sendsInSecondThread; i++) { + systemProducer.send(source1, ome2); + } + } + }; + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + systemProducer.stop(); + verify(mockAzureWriter1, times(sendsInFirstThread)).write(ome1); + verify(mockAzureWriter1, times(sendsInSecondThread)).write(ome2); + } + + @Test + public void testMutipleThreadFlushToSingleWriter() throws Exception { + String source1 = "FAKE_SOURCE_1"; + + AzureBlobWriter mockAzureWriter1 = mock(AzureBlobWriter.class); + doNothing().when(mockAzureWriter1).close(); + + AzureBlobConfig azureBlobConfig = new AzureBlobConfig(getBasicConfigs()); + + AzureBlobSystemProducer systemProducer = spy(new AzureBlobSystemProducer(SYSTEM_NAME, azureBlobConfig, mockMetricsRegistry)); + // bypass Azure connection setup + doNothing().when(systemProducer).setupAzureContainer(anyString(), anyString()); + + doReturn(mockAzureWriter1).when(systemProducer).createNewWriter(anyString(), any()); + + systemProducer.register(source1); + systemProducer.start(); + systemProducer.send(source1, ome); //to create writer + Thread t1 = new Thread() { + @Override + public void run() { + systemProducer.flush(source1); + } + }; + Thread t2 = new Thread() { + @Override + public void run() { + systemProducer.flush(source1); + } + }; + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + systemProducer.stop(); + // systemProducer.flush called twice but first flush clears the writer map of the source. + // hence, writer.flush and close called only once. + verify(mockAzureWriter1).flush(); + verify(mockAzureWriter1).close(); + } + + private Thread sendFlushInThread(String source, OutgoingMessageEnvelope ome, AzureBlobSystemProducer systemProducer, + int numberOfSends) { + Thread t = new Thread() { + @Override + public void run() { + try { + for (int i = 0; i < numberOfSends; i++) { + systemProducer.send(source, ome); + } + systemProducer.flush(source); + } catch (Exception e) { + throw e; + } + } + }; + return t; + } + + private OutgoingMessageEnvelope createOME(String streamName) { + SystemStream systemStream = new SystemStream(SYSTEM_NAME, streamName); + DummyPageViewEvent record = new DummyPageViewEvent(); + return new OutgoingMessageEnvelope(systemStream, record); + } + + private OutgoingMessageEnvelope createAnotherOME(String streamName) { + SystemStream systemStream = new SystemStream(SYSTEM_NAME, streamName); + AnotherDummyPageViewEvent record = new AnotherDummyPageViewEvent(); + return new OutgoingMessageEnvelope(systemStream, record); + } + + private class DummyPageViewEvent extends org.apache.avro.specific.SpecificRecordBase + implements org.apache.avro.specific.SpecificRecord { + public final org.apache.avro.Schema schema = org.apache.avro.Schema.parse( + "{\"type\":\"record\",\"name\":\"DummyPageViewEvent\",\"namespace\":\"org.apache.samza.events\",\"fields\":[]}"); + + public org.apache.avro.Schema getSchema() { + return schema; + } + + public java.lang.Object get(int field) { + return null; + } + + public void put(int field, Object value) {} + } + + private class AnotherDummyPageViewEvent extends org.apache.avro.specific.SpecificRecordBase + implements org.apache.avro.specific.SpecificRecord { + public final org.apache.avro.Schema schema = org.apache.avro.Schema.parse( + "{\"type\":\"record\",\"name\":\"AnotherDummyPageViewEvent\",\"namespace\":\"org.apache.samza.events\",\"fields\":[]}"); + + public org.apache.avro.Schema getSchema() { + return schema; + } + + public java.lang.Object get(int field) { + return null; + } + + public void put(int field, Object value) {} + } + + private Config getBasicConfigs() { + Map bareConfigs = new HashMap<>(); + bareConfigs.put(String.format(AzureBlobConfig.SYSTEM_AZURE_ACCOUNT_NAME, SYSTEM_NAME), ACCOUNT_NAME); + bareConfigs.put(String.format(AzureBlobConfig.SYSTEM_AZURE_ACCOUNT_KEY, SYSTEM_NAME), ACCOUNT_KEY); + bareConfigs.put(String.format(AzureBlobConfig.SYSTEM_CLOSE_TIMEOUT_MS, SYSTEM_NAME), "1000"); + Config config = new MapConfig(bareConfigs); + return config; + } +} \ No newline at end of file From 09ac78c34308b4283de6f36aef5a9355e6b1d41c Mon Sep 17 00:00:00 2001 From: shanthoosh Date: Fri, 17 Jan 2020 10:19:34 -0800 Subject: [PATCH 43/49] Fix the RocksDB TTL type conversion in change log properties generation. (#1254) --- .../org/apache/samza/config/KafkaConfig.scala | 2 +- .../apache/samza/config/TestKafkaConfig.scala | 19 +++++++++++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala index 3b5f5f3d89..69a99664c1 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala @@ -324,7 +324,7 @@ class KafkaConfig(config: Config) extends ScalaMapConfig(config) { // - Set topic TTL to be the same as RocksDB TTL Option(config.get("stores.%s.rocksdb.ttl.ms" format name)) match { case Some(rocksDbTtl) => - if (!rocksDbTtl.isEmpty && rocksDbTtl.toInt < 0) { + if (!rocksDbTtl.isEmpty && rocksDbTtl.toLong < 0) { kafkaChangeLogProperties.setProperty("cleanup.policy", "compact") kafkaChangeLogProperties.setProperty("max.message.bytes", getChangelogStreamMaxMessageByte(name)) } else if (!config.containsKey("stores.%s.changelog.kafka.cleanup.policy" format name)) { diff --git a/samza-kafka/src/test/scala/org/apache/samza/config/TestKafkaConfig.scala b/samza-kafka/src/test/scala/org/apache/samza/config/TestKafkaConfig.scala index 00b103d817..64b476b79a 100644 --- a/samza-kafka/src/test/scala/org/apache/samza/config/TestKafkaConfig.scala +++ b/samza-kafka/src/test/scala/org/apache/samza/config/TestKafkaConfig.scala @@ -104,6 +104,25 @@ class TestKafkaConfig { assertEquals("86400000", kafkaProperties.getProperty("delete.retention.ms")) } + @Test + def testChangeLogPropertiesShouldReturnCorrectTopicConfigurationForLargeTTLStores(): Unit = { + val props = new Properties + props.setProperty(KAFKA_PRODUCER_PROPERTY_PREFIX + "bootstrap.servers", "localhost:9092") + props.setProperty("systems." + SYSTEM_NAME + ".consumer.zookeeper.connect", "localhost:2181/") + props.setProperty(JobConfig.JOB_NAME, "jobName") + + props.setProperty("stores.test1.changelog", "kafka.mychangelog1") + // Set the RocksDB TTL to be 28 days. + props.setProperty("stores.test1.rocksdb.ttl.ms", "2419200000") + + val mapConfig = new MapConfig(props.asScala.asJava) + val kafkaConfig = new KafkaConfig(mapConfig) + val kafkaProperties = kafkaConfig.getChangelogKafkaProperties("test1") + assertEquals("delete", kafkaProperties.getProperty("cleanup.policy")) + assertEquals("536870912", kafkaProperties.getProperty("segment.bytes")) + assertEquals("86400000", kafkaProperties.getProperty("delete.retention.ms")) + } + @Test def testChangeLogPropertiesShouldReturnCorrectTopicConfigurationForStoresWithEmptyRocksDBTTL(): Unit = { val props = new Properties From c780c1af28f23b2708cc1b6d66ef1bb44247ae6a Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Fri, 17 Jan 2020 14:02:44 -0800 Subject: [PATCH 44/49] SAMZA-2409: Update YarnJob to construct job submission env variable (#1246) --- .../org/apache/samza/job/yarn/YarnJob.scala | 12 ++++++++--- .../apache/samza/job/yarn/TestYarnJob.java | 20 +++++++++++++++++++ 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJob.scala b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJob.scala index ae20a29a0c..43e6a7c659 100644 --- a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJob.scala +++ b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/YarnJob.scala @@ -173,10 +173,16 @@ object YarnJob extends Logging { @VisibleForTesting private[yarn] def buildEnvironment(config: Config, yarnConfig: YarnConfig, jobConfig: JobConfig): Map[String, String] = { - val coordinatorSystemConfig = CoordinatorStreamUtil.buildCoordinatorStreamConfig(config) val envMapBuilder = Map.newBuilder[String, String] - envMapBuilder += ShellCommandConfig.ENV_COORDINATOR_SYSTEM_CONFIG -> - Util.envVarEscape(SamzaObjectMapper.getObjectMapper.writeValueAsString(coordinatorSystemConfig)) + if (jobConfig.getConfigLoaderFactory.isPresent) { + envMapBuilder += ShellCommandConfig.ENV_SUBMISSION_CONFIG -> + Util.envVarEscape(SamzaObjectMapper.getObjectMapper.writeValueAsString(config)) + } else { + // TODO SAMZA-2432: Clean this up once SAMZA-2405 is completed when legacy flow is removed. + val coordinatorSystemConfig = CoordinatorStreamUtil.buildCoordinatorStreamConfig(config) + envMapBuilder += ShellCommandConfig.ENV_COORDINATOR_SYSTEM_CONFIG -> + Util.envVarEscape(SamzaObjectMapper.getObjectMapper.writeValueAsString(coordinatorSystemConfig)) + } envMapBuilder += ShellCommandConfig.ENV_JAVA_OPTS -> Util.envVarEscape(yarnConfig.getAmOpts) val clusterBasedJobCoordinatorDependencyIsolationEnabled = jobConfig.getClusterBasedJobCoordinatorDependencyIsolationEnabled diff --git a/samza-yarn/src/test/java/org/apache/samza/job/yarn/TestYarnJob.java b/samza-yarn/src/test/java/org/apache/samza/job/yarn/TestYarnJob.java index c10c890a9c..daa719bcf4 100644 --- a/samza-yarn/src/test/java/org/apache/samza/job/yarn/TestYarnJob.java +++ b/samza-yarn/src/test/java/org/apache/samza/job/yarn/TestYarnJob.java @@ -111,4 +111,24 @@ public void testBuildEnvironmentWithAMJavaHome() throws IOException { assertEquals(expected, JavaConverters.mapAsJavaMapConverter( YarnJob$.MODULE$.buildEnvironment(config, new YarnConfig(config), new JobConfig(config))).asJava()); } + + @Test + public void testBuildJobSubmissionEnvironment() throws IOException { + Config config = new MapConfig(new ImmutableMap.Builder() + .put(JobConfig.JOB_NAME, "jobName") + .put(JobConfig.JOB_ID, "jobId") + .put(JobConfig.CONFIG_LOADER_FACTORY, "org.apache.samza.config.loaders.PropertiesConfigLoaderFactory") + .put(YarnConfig.AM_JVM_OPTIONS, "") + .put(JobConfig.CLUSTER_BASED_JOB_COORDINATOR_DEPENDENCY_ISOLATION_ENABLED, "true") + .build()); + String expectedSubmissionConfig = Util.envVarEscape(SamzaObjectMapper.getObjectMapper() + .writeValueAsString(config)); + Map expected = ImmutableMap.of( + ShellCommandConfig.ENV_SUBMISSION_CONFIG(), expectedSubmissionConfig, + ShellCommandConfig.ENV_JAVA_OPTS(), "", + ShellCommandConfig.ENV_CLUSTER_BASED_JOB_COORDINATOR_DEPENDENCY_ISOLATION_ENABLED(), "true", + ShellCommandConfig.ENV_APPLICATION_LIB_DIR(), "./__package/lib"); + assertEquals(expected, JavaConverters.mapAsJavaMapConverter( + YarnJob$.MODULE$.buildEnvironment(config, new YarnConfig(config), new JobConfig(config))).asJava()); + } } \ No newline at end of file From cfcec2dc0eb579c7dc4560d3afdcae27a51a46a7 Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Fri, 17 Jan 2020 14:03:44 -0800 Subject: [PATCH 45/49] SAMZA-2408: Update RemoteApplicationRunner to submit job only (#1247) 1. Depending on the existence of job.config.loader.factory, RemoteApplicationRunner will alternatively only submit the job, without planning. 2. Late initialize RemoteJobPlanner only when needed to avoid executing user code in submitting only mode. --- .../runtime/RemoteApplicationRunner.java | 24 ++++++++++++------- .../runtime/TestRemoteApplicationRunner.java | 19 +++++++++++---- 2 files changed, 29 insertions(+), 14 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java index 9969b9d966..3af5db1eaf 100644 --- a/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java +++ b/samza-core/src/main/java/org/apache/samza/runtime/RemoteApplicationRunner.java @@ -22,8 +22,6 @@ import java.time.Duration; import java.util.List; import org.apache.samza.SamzaException; -import org.apache.samza.application.descriptors.ApplicationDescriptor; -import org.apache.samza.application.descriptors.ApplicationDescriptorImpl; import org.apache.samza.application.descriptors.ApplicationDescriptorUtil; import org.apache.samza.application.SamzaApplication; import org.apache.samza.config.Config; @@ -47,8 +45,8 @@ public class RemoteApplicationRunner implements ApplicationRunner { private static final Logger LOG = LoggerFactory.getLogger(RemoteApplicationRunner.class); private static final long DEFAULT_SLEEP_DURATION_MS = 2000; - private final ApplicationDescriptorImpl appDesc; - private final RemoteJobPlanner planner; + private final SamzaApplication app; + private final Config config; /** * Constructors a {@link RemoteApplicationRunner} to run the {@code app} with the {@code config}. @@ -57,13 +55,21 @@ public class RemoteApplicationRunner implements ApplicationRunner { * @param config configuration for the application */ public RemoteApplicationRunner(SamzaApplication app, Config config) { - this.appDesc = ApplicationDescriptorUtil.getAppDescriptor(app, config); - this.planner = new RemoteJobPlanner(appDesc); + this.app = app; + this.config = config; } @Override public void run(ExternalContext externalContext) { + if (new JobConfig(config).getConfigLoaderFactory().isPresent()) { + JobRunner runner = new JobRunner(JobPlanner.generateSingleJobConfig(config)); + runner.submit(); + return; + } + + // TODO SAMZA-2432: Clean this up once SAMZA-2405 is completed when legacy flow is removed. try { + JobPlanner planner = new RemoteJobPlanner(ApplicationDescriptorUtil.getAppDescriptor(app, config)); List jobConfigs = planner.prepareJobs(); if (jobConfigs.isEmpty()) { throw new SamzaException("No jobs to run."); @@ -85,7 +91,7 @@ public void kill() { // since currently we only support single actual remote job, we can get its status without // building the execution plan. try { - JobConfig jc = new JobConfig(JobPlanner.generateSingleJobConfig(appDesc.getConfig())); + JobConfig jc = new JobConfig(JobPlanner.generateSingleJobConfig(config)); LOG.info("Killing job {}", jc.getName()); JobRunner runner = new JobRunner(jc); runner.kill(); @@ -99,7 +105,7 @@ public ApplicationStatus status() { // since currently we only support single actual remote job, we can get its status without // building the execution plan try { - JobConfig jc = new JobConfig(JobPlanner.generateSingleJobConfig(appDesc.getConfig())); + JobConfig jc = new JobConfig(JobPlanner.generateSingleJobConfig(config)); return getApplicationStatus(jc); } catch (Throwable t) { throw new SamzaException("Failed to get status for application", t); @@ -113,7 +119,7 @@ public void waitForFinish() { @Override public boolean waitForFinish(Duration timeout) { - JobConfig jobConfig = new JobConfig(JobPlanner.generateSingleJobConfig(appDesc.getConfig())); + JobConfig jobConfig = new JobConfig(JobPlanner.generateSingleJobConfig(config)); boolean finished = true; long timeoutInMs = timeout.toMillis(); long startTimeInMs = System.currentTimeMillis(); diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java index 1464ece1c4..f38269482e 100644 --- a/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java +++ b/samza-core/src/test/java/org/apache/samza/runtime/TestRemoteApplicationRunner.java @@ -75,9 +75,20 @@ public void testWaitForFinishTimesout() { assertFalse("Application finished before the timeout.", finished); } + @Test + public void testRunWithConfigLoaderFactoryPresent() { + Map config = new HashMap<>(); + config.put(ApplicationConfig.APP_NAME, "test-app"); + config.put(JobConfig.CONFIG_LOADER_FACTORY, "org.apache.samza.config.loaders.PropertiesConfigLoaderFactory"); + config.put(JobConfig.STREAM_JOB_FACTORY_CLASS, MockStreamJobFactory.class.getName()); + runner = new RemoteApplicationRunner(null, new MapConfig(config)); + + runner.run(null); + } + @Test public void testGetStatus() { - Map m = new HashMap(); + Map m = new HashMap<>(); m.put(JobConfig.JOB_NAME, "jobName"); m.put(JobConfig.STREAM_JOB_FACTORY_CLASS, MockStreamJobFactory.class.getName()); @@ -101,8 +112,8 @@ public MockStreamJobFactory() { @Override public StreamJob getJob(final Config config) { - StreamJob streamJob = new StreamJob() { - JobConfig c = (JobConfig) config; + return new StreamJob() { + JobConfig c = new JobConfig(config); @Override public StreamJob submit() { @@ -137,8 +148,6 @@ public ApplicationStatus getStatus() { } } }; - - return streamJob; } } } From d7ed48f3a904215c7cf85a53a979ddbef813a38d Mon Sep 17 00:00:00 2001 From: Ke Wu Date: Fri, 17 Jan 2020 14:09:24 -0800 Subject: [PATCH 46/49] SAMZA-2410: Update ClusterBasedJobCoordinator config retrieval logic from loader. (#1248) 1. Based on the existence of ENV_SUBMISSION_CONFIG, ClusterBasedJobCoordinator will alternatively deserilize ENV_SUBMISSION_CONFIG and load full job config from config loader factory. 2. Execute planning, create diagnostics stream and persist full job config back to coordinator stream when loading full job config from config loader. --- .../ClusterBasedJobCoordinator.java | 174 +++++++++++++----- .../org/apache/samza/util/ConfigUtil.java | 43 +++++ .../samza/coordinator/JobModelManager.scala | 9 +- .../samza/util/CoordinatorStreamUtil.scala | 2 +- .../TestClusterBasedJobCoordinator.java | 54 +++++- .../org/apache/samza/util/TestConfigUtil.java | 30 +++ 6 files changed, 258 insertions(+), 54 deletions(-) diff --git a/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterBasedJobCoordinator.java b/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterBasedJobCoordinator.java index d9ca4d5ac6..d97a4a1e09 100644 --- a/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterBasedJobCoordinator.java +++ b/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterBasedJobCoordinator.java @@ -23,12 +23,18 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; +import org.apache.commons.lang3.StringUtils; import org.apache.samza.SamzaException; +import org.apache.samza.application.ApplicationUtil; +import org.apache.samza.application.descriptors.ApplicationDescriptor; +import org.apache.samza.application.descriptors.ApplicationDescriptorImpl; +import org.apache.samza.application.descriptors.ApplicationDescriptorUtil; import org.apache.samza.classloader.IsolatingClassLoaderFactory; import org.apache.samza.config.ClusterManagerConfig; import org.apache.samza.config.Config; @@ -47,10 +53,12 @@ import org.apache.samza.coordinator.metadatastore.CoordinatorStreamStore; import org.apache.samza.coordinator.metadatastore.NamespaceAwareCoordinatorStreamStore; import org.apache.samza.coordinator.stream.messages.SetChangelogMapping; +import org.apache.samza.execution.RemoteJobPlanner; import org.apache.samza.job.model.ContainerModel; import org.apache.samza.job.model.JobModel; import org.apache.samza.job.model.JobModelUtil; import org.apache.samza.job.model.TaskModel; +import org.apache.samza.metadatastore.MetadataStore; import org.apache.samza.metrics.JmxServer; import org.apache.samza.metrics.MetricsRegistryMap; import org.apache.samza.serializers.model.SamzaObjectMapper; @@ -59,6 +67,7 @@ import org.apache.samza.system.StreamMetadataCache; import org.apache.samza.system.SystemAdmins; import org.apache.samza.system.SystemStream; +import org.apache.samza.util.ConfigUtil; import org.apache.samza.util.CoordinatorStreamUtil; import org.apache.samza.util.DiagnosticsUtil; import org.apache.samza.util.SystemClock; @@ -95,7 +104,6 @@ public class ClusterBasedJobCoordinator { private final static String METRICS_SOURCE_NAME = "ApplicationMaster"; private final Config config; - private final ClusterManagerConfig clusterManagerConfig; /** * State to track container failures, host-processor mappings @@ -154,7 +162,7 @@ public class ClusterBasedJobCoordinator { * Metrics to track stats around container failures, needed containers etc. */ private final MetricsRegistryMap metrics; - private final CoordinatorStreamStore coordinatorStreamStore; + private final MetadataStore metadataStore; private final SystemAdmins systemAdmins; @@ -169,40 +177,38 @@ public class ClusterBasedJobCoordinator { volatile private Exception coordinatorException = null; /** - * Creates a new ClusterBasedJobCoordinator instance from a config. Invoke run() to actually - * run the jobcoordinator. + * Creates a new ClusterBasedJobCoordinator instance. + * Invoke run() to actually run the job coordinator. * - * @param coordinatorSystemConfig the coordinator stream config that can be used to read the - * {@link org.apache.samza.job.model.JobModel} from. + * @param metrics the registry for reporting metrics. + * @param metadataStore metadata store to hold metadata. + * @param fullJobConfig full job config. */ - public ClusterBasedJobCoordinator(Config coordinatorSystemConfig) { - metrics = new MetricsRegistryMap(); - - coordinatorStreamStore = new CoordinatorStreamStore(coordinatorSystemConfig, metrics); - coordinatorStreamStore.init(); - config = CoordinatorStreamUtil.readConfigFromCoordinatorStream(coordinatorStreamStore); - + private ClusterBasedJobCoordinator(MetricsRegistryMap metrics, MetadataStore metadataStore, Config fullJobConfig) { + this.metrics = metrics; + this.metadataStore = metadataStore; + this.config = fullJobConfig; // build a JobModelManager and ChangelogStreamManager and perform partition assignments. - changelogStreamManager = new ChangelogStreamManager(new NamespaceAwareCoordinatorStreamStore(coordinatorStreamStore, SetChangelogMapping.TYPE)); - jobModelManager = - JobModelManager.apply(config, changelogStreamManager.readPartitionMapping(), coordinatorStreamStore, metrics); + this.changelogStreamManager = new ChangelogStreamManager( + new NamespaceAwareCoordinatorStreamStore(metadataStore, SetChangelogMapping.TYPE)); + this.jobModelManager = + JobModelManager.apply(config, changelogStreamManager.readPartitionMapping(), metadataStore, metrics); - hasDurableStores = new StorageConfig(config).hasDurableStores(); - state = new SamzaApplicationState(jobModelManager); + this.hasDurableStores = new StorageConfig(config).hasDurableStores(); + this.state = new SamzaApplicationState(jobModelManager); // The systemAdmins should be started before partitionMonitor can be used. And it should be stopped when this coordinator is stopped. - systemAdmins = new SystemAdmins(config); - partitionMonitor = getPartitionCountMonitor(config, systemAdmins); + this.systemAdmins = new SystemAdmins(config); + this.partitionMonitor = getPartitionCountMonitor(config, systemAdmins); Set inputSystemStreams = JobModelUtil.getSystemStreams(jobModelManager.jobModel()); - inputStreamRegexMonitor = getInputRegexMonitor(config, systemAdmins, inputSystemStreams); + this.inputStreamRegexMonitor = getInputRegexMonitor(config, systemAdmins, inputSystemStreams); - clusterManagerConfig = new ClusterManagerConfig(config); - isJmxEnabled = clusterManagerConfig.getJmxEnabledOnJobCoordinator(); - - jobCoordinatorSleepInterval = clusterManagerConfig.getJobCoordinatorSleepInterval(); + ClusterManagerConfig clusterManagerConfig = new ClusterManagerConfig(config); + this.isJmxEnabled = clusterManagerConfig.getJmxEnabledOnJobCoordinator(); + this.jobCoordinatorSleepInterval = clusterManagerConfig.getJobCoordinatorSleepInterval(); // build a container process Manager - containerProcessManager = createContainerProcessManager(); + this.containerProcessManager = createContainerProcessManager(); } /** @@ -300,7 +306,7 @@ private void onShutDown() { inputStreamRegexMonitor.ifPresent(StreamRegexMonitor::stop); systemAdmins.stop(); containerProcessManager.stop(); - coordinatorStreamStore.close(); + metadataStore.close(); } catch (Throwable e) { LOG.error("Exception while stopping cluster based job coordinator", e); } @@ -386,8 +392,7 @@ public void onInputStreamsChanged(Set initialInputSet, Set cluste * {@link #main(String[])} so that it can be executed directly or from a separate classloader. */ private static void runClusterBasedJobCoordinator(String[] args) { - Config coordinatorSystemConfig; final String coordinatorSystemEnv = System.getenv(ShellCommandConfig.ENV_COORDINATOR_SYSTEM_CONFIG()); - try { - //Read and parse the coordinator system config. - LOG.info("Parsing coordinator system config {}", coordinatorSystemEnv); - coordinatorSystemConfig = - new MapConfig(SamzaObjectMapper.getObjectMapper().readValue(coordinatorSystemEnv, Config.class)); - LOG.info("Using the coordinator system config: {}.", coordinatorSystemConfig); - } catch (IOException e) { - LOG.error("Exception while reading coordinator stream config", e); - throw new SamzaException(e); + final String submissionEnv = System.getenv(ShellCommandConfig.ENV_SUBMISSION_CONFIG()); + + if (!StringUtils.isBlank(submissionEnv)) { + Config submissionConfig; + try { + //Read and parse the coordinator system config. + LOG.info("Parsing submission config {}", submissionEnv); + submissionConfig = + new MapConfig(SamzaObjectMapper.getObjectMapper().readValue(submissionEnv, Config.class)); + LOG.info("Using the submission config: {}.", submissionConfig); + } catch (IOException e) { + LOG.error("Exception while reading submission config", e); + throw new SamzaException(e); + } + + ClusterBasedJobCoordinator jc = createFromConfigLoader(submissionConfig); + jc.run(); + LOG.info("Finished running ClusterBasedJobCoordinator"); + } else { + // TODO: Clean this up once SAMZA-2405 is completed when legacy flow is removed. + Config coordinatorSystemConfig; + try { + //Read and parse the coordinator system config. + LOG.info("Parsing coordinator system config {}", coordinatorSystemEnv); + coordinatorSystemConfig = + new MapConfig(SamzaObjectMapper.getObjectMapper().readValue(coordinatorSystemEnv, Config.class)); + LOG.info("Using the coordinator system config: {}.", coordinatorSystemConfig); + } catch (IOException e) { + LOG.error("Exception while reading coordinator stream config", e); + throw new SamzaException(e); + } + ClusterBasedJobCoordinator jc = createFromMetadataStore(coordinatorSystemConfig); + jc.run(); + LOG.info("Finished running ClusterBasedJobCoordinator"); } - ClusterBasedJobCoordinator jc = new ClusterBasedJobCoordinator(coordinatorSystemConfig); - jc.run(); - LOG.info("Finished running ClusterBasedJobCoordinator"); + } + + /** + * Initialize {@link ClusterBasedJobCoordinator} with coordinator stream config, full job config will be fetched from + * coordinator stream. + * + * @param metadataStoreConfig to initialize {@link MetadataStore} + * @return {@link ClusterBasedJobCoordinator} + */ + // TODO SAMZA-2432: Clean this up once SAMZA-2405 is completed when legacy flow is removed. + @VisibleForTesting + static ClusterBasedJobCoordinator createFromMetadataStore(Config metadataStoreConfig) { + MetricsRegistryMap metrics = new MetricsRegistryMap(); + + CoordinatorStreamStore coordinatorStreamStore = new CoordinatorStreamStore(metadataStoreConfig, metrics); + coordinatorStreamStore.init(); + Config config = CoordinatorStreamUtil.readConfigFromCoordinatorStream(coordinatorStreamStore); + + return new ClusterBasedJobCoordinator(metrics, coordinatorStreamStore, config); + } + + /** + * Initialize {@link ClusterBasedJobCoordinator} with submission config, full job config will be fetched using + * specified {@link org.apache.samza.config.ConfigLoaderFactory} + * + * @param submissionConfig specifies {@link org.apache.samza.config.ConfigLoaderFactory} + * @return {@link ClusterBasedJobCoordinator} + */ + @VisibleForTesting + static ClusterBasedJobCoordinator createFromConfigLoader(Config submissionConfig) { + JobConfig jobConfig = new JobConfig(submissionConfig); + + if (!jobConfig.getConfigLoaderFactory().isPresent()) { + throw new SamzaException(JobConfig.CONFIG_LOADER_FACTORY + " is required to initialize job coordinator from config loader"); + } + + MetricsRegistryMap metrics = new MetricsRegistryMap(); + // load full job config with ConfigLoader + Config originalConfig = ConfigUtil.loadConfig(submissionConfig); + + // Execute planning + ApplicationDescriptorImpl + appDesc = ApplicationDescriptorUtil.getAppDescriptor(ApplicationUtil.fromConfig(originalConfig), originalConfig); + RemoteJobPlanner planner = new RemoteJobPlanner(appDesc); + List jobConfigs = planner.prepareJobs(); + + if (jobConfigs.size() != 1) { + throw new SamzaException("Only support single remote job is supported."); + } + + Config config = jobConfigs.get(0); + + // This needs to be consistent with RemoteApplicationRunner#run where JobRunner#submit to be called instead of JobRunner#run + CoordinatorStreamUtil.writeConfigToCoordinatorStream(config, true); + DiagnosticsUtil.createDiagnosticsStream(config); + MetadataStore metadataStore = new CoordinatorStreamStore(CoordinatorStreamUtil.buildCoordinatorStreamConfig(config), metrics); + metadataStore.init(); + + return new ClusterBasedJobCoordinator( + metrics, + metadataStore, + config); } } diff --git a/samza-core/src/main/java/org/apache/samza/util/ConfigUtil.java b/samza-core/src/main/java/org/apache/samza/util/ConfigUtil.java index 7d86bf52a0..7206fe2848 100644 --- a/samza-core/src/main/java/org/apache/samza/util/ConfigUtil.java +++ b/samza-core/src/main/java/org/apache/samza/util/ConfigUtil.java @@ -19,11 +19,17 @@ package org.apache.samza.util; +import java.util.HashMap; +import java.util.Map; import java.util.Optional; import org.apache.samza.SamzaException; import org.apache.samza.config.Config; +import org.apache.samza.config.ConfigException; +import org.apache.samza.config.ConfigLoader; +import org.apache.samza.config.ConfigLoaderFactory; import org.apache.samza.config.ConfigRewriter; import org.apache.samza.config.JobConfig; +import org.apache.samza.config.MapConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -67,4 +73,41 @@ public static Config applyRewriter(Config config, String rewriterName) { LOG.info("Re-writing config with {}", rewriter); return rewriter.rewrite(rewriterName, config); } + + /** + * Load full job config with {@link ConfigLoaderFactory} when present. + * + * @param original config + * @return full job config + */ + public static Config loadConfig(Config original) { + JobConfig jobConfig = new JobConfig(original); + + if (!jobConfig.getConfigLoaderFactory().isPresent()) { + throw new ConfigException("Missing key " + JobConfig.CONFIG_LOADER_FACTORY + "."); + } + + ConfigLoaderFactory factory = ReflectionUtil.getObj(jobConfig.getConfigLoaderFactory().get(), ConfigLoaderFactory.class); + ConfigLoader loader = factory.getLoader(original.subset(ConfigLoaderFactory.CONFIG_LOADER_PROPERTIES_PREFIX)); + // overrides config loaded with original config, which may contain overridden values. + return rewriteConfig(override(loader.getConfig(), original)); + } + + /** + * Overrides original config with overridden values. + * + * @param original config to be overridden. + * @param overrides overridden values. + * @return the overridden config. + */ + @SafeVarargs + private static Config override(Config original, Map... overrides) { + Map map = new HashMap<>(original); + + for (Map override : overrides) { + map.putAll(override); + } + + return new MapConfig(map); + } } diff --git a/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala b/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala index 07055de1fc..b8c18fe768 100644 --- a/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala +++ b/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala @@ -42,6 +42,7 @@ import org.apache.samza.job.model.ContainerModel import org.apache.samza.job.model.JobModel import org.apache.samza.job.model.TaskMode import org.apache.samza.job.model.TaskModel +import org.apache.samza.metadatastore.MetadataStore import org.apache.samza.metrics.MetricsRegistry import org.apache.samza.metrics.MetricsRegistryMap import org.apache.samza.runtime.LocationId @@ -79,13 +80,13 @@ object JobModelManager extends Logging { * @return the instantiated {@see JobModelManager}. */ def apply(config: Config, changelogPartitionMapping: util.Map[TaskName, Integer], - coordinatorStreamStore: CoordinatorStreamStore, + metadataStore: MetadataStore, metricsRegistry: MetricsRegistry = new MetricsRegistryMap()): JobModelManager = { // Instantiate the respective metadata store util classes which uses the same coordinator metadata store. - val localityManager = new LocalityManager(new NamespaceAwareCoordinatorStreamStore(coordinatorStreamStore, SetContainerHostMapping.TYPE)) - val taskAssignmentManager = new TaskAssignmentManager(new NamespaceAwareCoordinatorStreamStore(coordinatorStreamStore, SetTaskContainerMapping.TYPE), new NamespaceAwareCoordinatorStreamStore(coordinatorStreamStore, SetTaskModeMapping.TYPE)) - val taskPartitionAssignmentManager = new TaskPartitionAssignmentManager(new NamespaceAwareCoordinatorStreamStore(coordinatorStreamStore, SetTaskPartitionMapping.TYPE)) + val localityManager = new LocalityManager(new NamespaceAwareCoordinatorStreamStore(metadataStore, SetContainerHostMapping.TYPE)) + val taskAssignmentManager = new TaskAssignmentManager(new NamespaceAwareCoordinatorStreamStore(metadataStore, SetTaskContainerMapping.TYPE), new NamespaceAwareCoordinatorStreamStore(metadataStore, SetTaskModeMapping.TYPE)) + val taskPartitionAssignmentManager = new TaskPartitionAssignmentManager(new NamespaceAwareCoordinatorStreamStore(metadataStore, SetTaskPartitionMapping.TYPE)) val systemAdmins = new SystemAdmins(config) try { diff --git a/samza-core/src/main/scala/org/apache/samza/util/CoordinatorStreamUtil.scala b/samza-core/src/main/scala/org/apache/samza/util/CoordinatorStreamUtil.scala index f108387a2c..c9aad0b8bd 100644 --- a/samza-core/src/main/scala/org/apache/samza/util/CoordinatorStreamUtil.scala +++ b/samza-core/src/main/scala/org/apache/samza/util/CoordinatorStreamUtil.scala @@ -41,7 +41,7 @@ object CoordinatorStreamUtil extends Logging { */ def buildCoordinatorStreamConfig(config: Config): MapConfig = { val jobConfig = new JobConfig(config) - val buildConfigFactory = jobConfig.getCoordinatorStreamFactory() + val buildConfigFactory = jobConfig.getCoordinatorStreamFactory val coordinatorSystemConfig = Class.forName(buildConfigFactory).newInstance().asInstanceOf[CoordinatorStreamConfigFactory].buildCoordinatorStreamConfig(config) new MapConfig(coordinatorSystemConfig); diff --git a/samza-core/src/test/java/org/apache/samza/clustermanager/TestClusterBasedJobCoordinator.java b/samza-core/src/test/java/org/apache/samza/clustermanager/TestClusterBasedJobCoordinator.java index 8c51c642f0..787edf217a 100644 --- a/samza-core/src/test/java/org/apache/samza/clustermanager/TestClusterBasedJobCoordinator.java +++ b/samza-core/src/test/java/org/apache/samza/clustermanager/TestClusterBasedJobCoordinator.java @@ -21,21 +21,29 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import org.apache.samza.Partition; import org.apache.samza.SamzaException; +import org.apache.samza.application.MockStreamApplication; +import org.apache.samza.config.ApplicationConfig; import org.apache.samza.config.Config; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; +import org.apache.samza.config.loaders.PropertiesConfigLoaderFactory; import org.apache.samza.coordinator.StreamPartitionCountMonitor; +import org.apache.samza.coordinator.metadatastore.CoordinatorStreamStore; import org.apache.samza.coordinator.stream.CoordinatorStreamSystemProducer; import org.apache.samza.coordinator.stream.MockCoordinatorStreamSystemFactory; +import org.apache.samza.execution.RemoteJobPlanner; import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.metrics.MetricsRegistryMap; import org.apache.samza.startpoint.StartpointManager; import org.apache.samza.system.MockSystemFactory; import org.apache.samza.system.SystemStream; import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.util.ConfigUtil; import org.apache.samza.util.CoordinatorStreamUtil; import org.junit.After; import org.junit.Before; @@ -51,7 +59,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; import static org.mockito.AdditionalMatchers.aryEq; -import static org.mockito.Matchers.anyObject; +import static org.mockito.Matchers.*; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; @@ -59,16 +67,21 @@ import static org.mockito.Mockito.when; import static org.powermock.api.mockito.PowerMockito.mock; import static org.powermock.api.mockito.PowerMockito.verifyPrivate; +import static org.powermock.api.mockito.PowerMockito.verifyNew; /** * Tests for {@link ClusterBasedJobCoordinator} */ @RunWith(PowerMockRunner.class) -@PrepareForTest({CoordinatorStreamUtil.class, ClusterBasedJobCoordinator.class}) +@PrepareForTest({ + CoordinatorStreamUtil.class, + ClusterBasedJobCoordinator.class, + CoordinatorStreamStore.class, + RemoteJobPlanner.class}) public class TestClusterBasedJobCoordinator { - Map configMap; + private Map configMap; @Before public void setUp() { @@ -106,7 +119,7 @@ public void testPartitionCountMonitorWithDurableStates() { CoordinatorStreamSystemProducer producer = new CoordinatorStreamSystemProducer(config, mock(MetricsRegistry.class)); producer.writeConfig("test-job", config); - ClusterBasedJobCoordinator clusterCoordinator = new ClusterBasedJobCoordinator(config); + ClusterBasedJobCoordinator clusterCoordinator = ClusterBasedJobCoordinator.createFromMetadataStore(config); // change the input system stream metadata MockSystemFactory.MSG_QUEUES.put(new SystemStreamPartition("kafka", "topic1", new Partition(1)), new ArrayList<>()); @@ -126,7 +139,7 @@ public void testPartitionCountMonitorWithoutDurableStates() { CoordinatorStreamSystemProducer producer = new CoordinatorStreamSystemProducer(config, mock(MetricsRegistry.class)); producer.writeConfig("test-job", config); - ClusterBasedJobCoordinator clusterCoordinator = new ClusterBasedJobCoordinator(config); + ClusterBasedJobCoordinator clusterCoordinator = ClusterBasedJobCoordinator.createFromMetadataStore(config); // change the input system stream metadata MockSystemFactory.MSG_QUEUES.put(new SystemStreamPartition("kafka", "topic1", new Partition(1)), new ArrayList<>()); @@ -144,7 +157,7 @@ public void testVerifyStartpointManagerFanOut() throws IOException { Config config = new MapConfig(configMap); MockitoException stopException = new MockitoException("Stop"); - ClusterBasedJobCoordinator clusterCoordinator = Mockito.spy(new ClusterBasedJobCoordinator(config)); + ClusterBasedJobCoordinator clusterCoordinator = Mockito.spy(ClusterBasedJobCoordinator.createFromMetadataStore(config)); ContainerProcessManager mockContainerProcessManager = mock(ContainerProcessManager.class); doReturn(true).when(mockContainerProcessManager).shouldShutdown(); StartpointManager mockStartpointManager = mock(StartpointManager.class); @@ -198,4 +211,33 @@ public void testRunWithClassLoader() throws Exception { // make sure runClusterBasedJobCoordinator only got called once verifyPrivate(ClusterBasedJobCoordinator.class).invoke("runClusterBasedJobCoordinator", new Object[]{aryEq(args)}); } + + @Test(expected = SamzaException.class) + public void testCreateFromConfigLoaderWithoutConfigLoaderFactory() { + ClusterBasedJobCoordinator.createFromConfigLoader(new MapConfig()); + } + + @Test + public void testCreateFromConfigLoader() throws Exception { + Map config = new HashMap<>(); + config.put(ApplicationConfig.APP_CLASS, MockStreamApplication.class.getCanonicalName()); + config.put(JobConfig.CONFIG_LOADER_FACTORY, PropertiesConfigLoaderFactory.class.getCanonicalName()); + config.put(PropertiesConfigLoaderFactory.CONFIG_LOADER_PROPERTIES_PREFIX + "path", + getClass().getResource("/test.properties").getPath()); + Config submissionConfig = new MapConfig(config); + JobConfig fullJobConfig = new JobConfig(ConfigUtil.loadConfig(submissionConfig)); + + RemoteJobPlanner mockJobPlanner = mock(RemoteJobPlanner.class); + CoordinatorStreamStore mockCoordinatorStreamStore = mock(CoordinatorStreamStore.class); + + PowerMockito.whenNew(ClusterBasedJobCoordinator.class).withAnyArguments().thenReturn(mock(ClusterBasedJobCoordinator.class)); + PowerMockito.doReturn(new MapConfig()).when(CoordinatorStreamUtil.class, "buildCoordinatorStreamConfig", any()); + PowerMockito.whenNew(CoordinatorStreamStore.class).withAnyArguments().thenReturn(mockCoordinatorStreamStore); + PowerMockito.whenNew(RemoteJobPlanner.class).withAnyArguments().thenReturn(mockJobPlanner); + when(mockJobPlanner.prepareJobs()).thenReturn(Collections.singletonList(fullJobConfig)); + + ClusterBasedJobCoordinator.createFromConfigLoader(submissionConfig); + + verifyNew(ClusterBasedJobCoordinator.class).withArguments(any(MetricsRegistryMap.class), eq(mockCoordinatorStreamStore), eq(fullJobConfig)); + } } diff --git a/samza-core/src/test/java/org/apache/samza/util/TestConfigUtil.java b/samza-core/src/test/java/org/apache/samza/util/TestConfigUtil.java index 218eb179fd..c005f37a35 100644 --- a/samza-core/src/test/java/org/apache/samza/util/TestConfigUtil.java +++ b/samza-core/src/test/java/org/apache/samza/util/TestConfigUtil.java @@ -24,9 +24,11 @@ import com.google.common.collect.ImmutableMap; import org.apache.samza.SamzaException; import org.apache.samza.config.Config; +import org.apache.samza.config.ConfigException; import org.apache.samza.config.ConfigRewriter; import org.apache.samza.config.JobConfig; import org.apache.samza.config.MapConfig; +import org.apache.samza.config.loaders.PropertiesConfigLoaderFactory; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -161,6 +163,34 @@ public void testApplyRewriterClassDoesNotExist() { assertEquals(expectedConfig, ConfigUtil.applyRewriter(new MapConfig(fullConfig), REWRITER_NAME)); } + @Test(expected = ConfigException.class) + public void testLoadConfigWithoutLoader() { + Map config = new HashMap<>(); + config.put(JobConfig.JOB_NAME, "new-test-job"); + + ConfigUtil.loadConfig(new MapConfig(config)); + } + + @Test + public void testLoadConfigWithOverridesAndRewrites() { + Map config = new HashMap<>(); + config.put(JobConfig.CONFIG_LOADER_FACTORY, PropertiesConfigLoaderFactory.class.getCanonicalName()); + config.put(JobConfig.JOB_NAME, "new-test-job"); + config.put(JobConfig.CONFIG_REWRITERS, REWRITER_NAME); + config.put(CONFIG_KEY, CONFIG_VALUE); + config.put(String.format(JobConfig.CONFIG_REWRITER_CLASS, REWRITER_NAME), NewPropertyRewriter.class.getName()); + config.put(PropertiesConfigLoaderFactory.CONFIG_LOADER_PROPERTIES_PREFIX + "path", getClass().getResource("/test.properties").getPath()); + + Config actual = ConfigUtil.loadConfig(new MapConfig(config)); + + assertEquals("org.apache.samza.job.MockJobFactory", actual.get("job.factory.class")); + assertEquals("bar", actual.get("foo")); + // overridden value + assertEquals("new-test-job", actual.get("job.name")); + // rewritten value + assertEquals(CONFIG_VALUE, actual.get(NEW_CONFIG_KEY)); + } + /** * Adds a new config entry for the key {@link #NEW_CONFIG_KEY} which has the same value as {@link #CONFIG_KEY}. */ From dec12ff21f30eb5f4bf96459ffe43ffb7c4e4c48 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Sun, 19 Jan 2020 11:51:55 -0800 Subject: [PATCH 47/49] Removing additional method --- .../org/apache/samza/metrics/MetricsRegistry.java | 11 ----------- .../org/apache/samza/util/NoOpMetricsRegistry.java | 5 ----- .../samza/system/eventhub/TestMetricsRegistry.java | 7 ------- .../java/org/apache/samza/metrics/MetricGroup.java | 12 +----------- .../org/apache/samza/metrics/MetricsHelper.scala | 6 ------ .../apache/samza/metrics/MetricsRegistryMap.scala | 13 +++---------- .../samza/storage/kv/RocksDbKeyValueStore.scala | 2 +- .../samza/storage/kv/TestRocksDbKeyValueStore.scala | 4 ++-- .../samza/sql/util/TestMetricsRegistryImpl.java | 7 ------- 9 files changed, 7 insertions(+), 60 deletions(-) diff --git a/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java b/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java index f5a4a438c0..5a00d01868 100644 --- a/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java +++ b/samza-api/src/main/java/org/apache/samza/metrics/MetricsRegistry.java @@ -64,17 +64,6 @@ public interface MetricsRegistry { */ Gauge newGauge(String group, Gauge value); - /** - * Register an existing {@link org.apache.samza.metrics.Gauge} - * @param group Group for this Gauge - * @param value Initial value for the Gauge - * @param Type the Gauge will be wrapping - * @param overrideExistingGauge Overwrite any existing gauges present for the same group and gauge - * @return Gauge was registered - */ - Gauge newGauge(String group, Gauge value, Boolean overrideExistingGauge); - - /** * Create and Register a new {@link org.apache.samza.metrics.Timer} * @param group Group for this Timer diff --git a/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java b/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java index e2519f171b..068fe72bb1 100644 --- a/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java +++ b/samza-api/src/main/java/org/apache/samza/util/NoOpMetricsRegistry.java @@ -50,11 +50,6 @@ public Gauge newGauge(String group, Gauge gauge) { return gauge; } - @Override - public Gauge newGauge(String group, Gauge gauge, Boolean overrideExistingGauge) { - return gauge; - } - @Override public Timer newTimer(String group, String name) { return new Timer(name); diff --git a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java index 534b70563d..48f6edda29 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java +++ b/samza-azure/src/test/java/org/apache/samza/system/eventhub/TestMetricsRegistry.java @@ -77,13 +77,6 @@ public Gauge newGauge(String group, Gauge value) { return value; } - @Override - public Gauge newGauge(String group, Gauge gauge, Boolean overrideExistingGauge) { - gauges.putIfAbsent(group, new ArrayList<>()); - gauges.get(group).add(gauge); - return gauge; - } - @Override public Timer newTimer(String group, String name) { return null; diff --git a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java index dec56c4356..015ab3aca1 100644 --- a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java +++ b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java @@ -54,17 +54,7 @@ public Gauge newGauge(String name, T value) { * it from another thread. */ public Gauge newGauge(String name, final ValueFunction valueFunc) { - return this.newGauge(name, valueFunc, false); - } - - // TODO SAMZA-2417 make overwriting the default behavior - public Gauge newGauge(String name, final ValueFunction valueFunc, Boolean overrideExistingGauge) { - return registry.newGauge(groupName, new Gauge((prefix + name).toLowerCase(), valueFunc.getValue()) { - @Override - public T getValue() { - return valueFunc.getValue(); - } - }, overrideExistingGauge); + return this.newGauge(name, valueFunc); } public Timer newTimer(String name) { diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala index cf19c3fc64..6e1fb8cf6c 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsHelper.scala @@ -51,12 +51,6 @@ trait MetricsHelper { }) } - def newGauge[T](name: String, value: () => T, overrideExistingGauge : Boolean) = { - metricGroup.newGauge(name, new ValueFunction[T] { - override def getValue = value() - }, overrideExistingGauge) - } - /** * Returns a prefix for metric names. */ diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala index 5a291fe7a4..3d60e0b1d2 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala @@ -52,17 +52,10 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with } def newGauge[T](group: String, gauge: Gauge[T]) = { - newGauge(group, gauge, false) - } - - def newGauge[T](group: String, gauge: Gauge[T], overrideExistingGauge: lang.Boolean) = { - if (overrideExistingGauge) { - debug("Updating existing gauge %s %s %s" format (group, gauge.getName, gauge)) - putAndGetGroup(group).put(gauge.getName, gauge) - } else { - debug("Adding new gauge %s %s %s." format (group, gauge.getName, gauge)) - putAndGetGroup(group).putIfAbsent(gauge.getName, gauge) + if (putAndGetGroup(group).containsKey(gauge.getName)) { + info("Updating existing gauge %s %s %s" format (group, gauge.getName, gauge)) } + putAndGetGroup(group).put(gauge.getName, gauge) val realGauge = metrics.get(group).get(gauge.getName).asInstanceOf[Gauge[T]] listeners.foreach(_.onGauge(group, realGauge)) realGauge diff --git a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala index 40fd0258ce..300177af6a 100644 --- a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala +++ b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala @@ -98,7 +98,7 @@ object RocksDbKeyValueStore extends Logging { rocksDb.getProperty(property) } else { "0" - }, true // overwrite-gauge = true, to accommodate for store-close and re-open (after bulk-load) + } )) rocksDb diff --git a/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala b/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala index f06797558c..2a4f44eafc 100644 --- a/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala +++ b/samza-kv-rocksdb/src/test/scala/org/apache/samza/storage/kv/TestRocksDbKeyValueStore.scala @@ -221,13 +221,13 @@ class TestRocksDbKeyValueStore val readWriteStoreMetricValue = "10" // Metric during bulk-load/bootstrap - metrics.newGauge("estimate-num-keys", () => bulkloadStoreMetricValue, true) + metrics.newGauge("estimate-num-keys", () => bulkloadStoreMetricValue) assert(registry.getGroup("org.apache.samza.storage.kv.KeyValueStoreMetrics"). get("dbstore-estimate-num-keys").asInstanceOf[Gauge[String]].getValue.eq("100")) // Bulk-load complete, new store in read-write mode - metrics.newGauge("estimate-num-keys", () => readWriteStoreMetricValue.toString, true) + metrics.newGauge("estimate-num-keys", () => readWriteStoreMetricValue.toString) assert(registry.getGroup("org.apache.samza.storage.kv.KeyValueStoreMetrics"). get("dbstore-estimate-num-keys").asInstanceOf[Gauge[String]].getValue.eq("10")) diff --git a/samza-sql/src/test/java/org/apache/samza/sql/util/TestMetricsRegistryImpl.java b/samza-sql/src/test/java/org/apache/samza/sql/util/TestMetricsRegistryImpl.java index b594a527ce..9a4b8c9eb1 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/util/TestMetricsRegistryImpl.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/util/TestMetricsRegistryImpl.java @@ -97,13 +97,6 @@ public Gauge newGauge(String group, Gauge gauge) { return gauge; } - @Override - public Gauge newGauge(String group, Gauge gauge, Boolean overrideExistingGauge) { - gauges.putIfAbsent(group, new ArrayList<>()); - gauges.get(group).add(gauge); - return gauge; - } - /** * retrieves the Map of Gauges * @return gauges From e9bd445323f984de0b86c122dd5d5432db1f4879 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Mon, 20 Jan 2020 21:18:06 -0800 Subject: [PATCH 48/49] Undoing MetricGroup change --- .../main/java/org/apache/samza/metrics/MetricGroup.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java index 015ab3aca1..53526d86c4 100644 --- a/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java +++ b/samza-core/src/main/java/org/apache/samza/metrics/MetricGroup.java @@ -54,7 +54,12 @@ public Gauge newGauge(String name, T value) { * it from another thread. */ public Gauge newGauge(String name, final ValueFunction valueFunc) { - return this.newGauge(name, valueFunc); + return registry.newGauge(groupName, new Gauge((prefix + name).toLowerCase(), valueFunc.getValue()) { + @Override + public T getValue() { + return valueFunc.getValue(); + } + }); } public Timer newTimer(String name) { From 1f50662be9f73ac26f5a1f14117978c05a8ddb16 Mon Sep 17 00:00:00 2001 From: Ray Matharu Date: Tue, 21 Jan 2020 13:05:12 -0800 Subject: [PATCH 49/49] Minor --- .../scala/org/apache/samza/metrics/MetricsRegistryMap.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala index 3d60e0b1d2..9751f688c8 100644 --- a/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala +++ b/samza-core/src/main/scala/org/apache/samza/metrics/MetricsRegistryMap.scala @@ -19,8 +19,6 @@ package org.apache.samza.metrics -import java.lang - import org.apache.samza.util.Logging import java.util.concurrent.ConcurrentHashMap @@ -53,7 +51,7 @@ class MetricsRegistryMap(val name: String) extends ReadableMetricsRegistry with def newGauge[T](group: String, gauge: Gauge[T]) = { if (putAndGetGroup(group).containsKey(gauge.getName)) { - info("Updating existing gauge %s %s %s" format (group, gauge.getName, gauge)) + debug("Updating existing gauge %s %s %s" format (group, gauge.getName, gauge)) } putAndGetGroup(group).put(gauge.getName, gauge) val realGauge = metrics.get(group).get(gauge.getName).asInstanceOf[Gauge[T]]