diff --git a/build.gradle b/build.gradle index a15a456bc2..a5f8cd3e6e 100644 --- a/build.gradle +++ b/build.gradle @@ -149,7 +149,7 @@ project(':samza-api') { } } -project(":samza-core_$scalaVersion") { +project(":samza-autoscaling_$scalaVersion") { apply plugin: 'scala' apply plugin: 'checkstyle' @@ -162,31 +162,25 @@ project(":samza-core_$scalaVersion") { sourceSets.main.java.srcDirs = [] sourceSets.test.java.srcDirs = [] - jar { - manifest { - attributes("Implementation-Version": "$version") - } - } - dependencies { compile project(':samza-api') - compile("com.101tec:zkclient:$zkClientVersion") { - exclude module: 'junit:junit' - } - compile "com.google.guava:guava:$guavaVersion" + compile project(":samza-core_$scalaVersion") + compile "org.scala-lang:scala-library:$scalaLibVersion" + compile "org.slf4j:slf4j-api:$slf4jVersion" compile "net.sf.jopt-simple:jopt-simple:$joptSimpleVersion" - compile "org.apache.commons:commons-collections4:$apacheCommonsCollections4Version" - compile "org.apache.commons:commons-lang3:$commonsLang3Version" compile "org.codehaus.jackson:jackson-mapper-asl:$jacksonVersion" compile "org.eclipse.jetty:jetty-webapp:$jettyVersion" - compile "org.scala-lang:scala-library:$scalaLibVersion" - compile "org.slf4j:slf4j-api:$slf4jVersion" - testCompile project(":samza-api").sourceSets.test.output + compile("org.apache.hadoop:hadoop-yarn-client:$yarnVersion") { + exclude module: 'slf4j-log4j12' + exclude module: 'servlet-api' + } + compile("org.apache.hadoop:hadoop-common:$yarnVersion") { + exclude module: 'slf4j-log4j12' + exclude module: 'servlet-api' + } + compile "org.apache.httpcomponents:httpclient:$httpClientVersion" testCompile "junit:junit:$junitVersion" testCompile "org.mockito:mockito-core:$mockitoVersion" - testCompile "org.powermock:powermock-api-mockito:$powerMockVersion" - testCompile "org.powermock:powermock-core:$powerMockVersion" - testCompile "org.powermock:powermock-module-junit4:$powerMockVersion" testCompile "org.scalatest:scalatest_$scalaVersion:$scalaTestVersion" } @@ -196,30 +190,6 @@ project(":samza-core_$scalaVersion") { } } -project(':samza-azure') { - apply plugin: 'java' - apply plugin: 'checkstyle' - - dependencies { - compile "com.microsoft.azure:azure-storage:5.3.1" - compile "com.microsoft.azure:azure-eventhubs:0.14.5" - compile "com.fasterxml.jackson.core:jackson-core:2.8.8" - compile "io.dropwizard.metrics:metrics-core:3.1.2" - compile project(':samza-api') - compile project(":samza-core_$scalaVersion") - compile "org.slf4j:slf4j-api:$slf4jVersion" - testCompile "junit:junit:$junitVersion" - testCompile "org.mockito:mockito-core:$mockitoVersion" - testCompile "org.powermock:powermock-api-mockito:$powerMockVersion" - testCompile "org.powermock:powermock-core:$powerMockVersion" - testCompile "org.powermock:powermock-module-junit4:$powerMockVersion" - } - checkstyle { - configFile = new File(rootDir, "checkstyle/checkstyle.xml") - toolVersion = "$checkstyleVersion" - } -} - project(':samza-aws') { apply plugin: 'java' apply plugin: 'checkstyle' @@ -252,8 +222,31 @@ project(':samza-aws') { } } +project(':samza-azure') { + apply plugin: 'java' + apply plugin: 'checkstyle' -project(":samza-autoscaling_$scalaVersion") { + dependencies { + compile "com.microsoft.azure:azure-storage:5.3.1" + compile "com.microsoft.azure:azure-eventhubs:0.14.5" + compile "com.fasterxml.jackson.core:jackson-core:2.8.8" + compile "io.dropwizard.metrics:metrics-core:3.1.2" + compile project(':samza-api') + compile project(":samza-core_$scalaVersion") + compile "org.slf4j:slf4j-api:$slf4jVersion" + testCompile "junit:junit:$junitVersion" + testCompile "org.mockito:mockito-core:$mockitoVersion" + testCompile "org.powermock:powermock-api-mockito:$powerMockVersion" + testCompile "org.powermock:powermock-core:$powerMockVersion" + testCompile "org.powermock:powermock-module-junit4:$powerMockVersion" + } + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + toolVersion = "$checkstyleVersion" + } +} + +project(":samza-core_$scalaVersion") { apply plugin: 'scala' apply plugin: 'checkstyle' @@ -266,25 +259,31 @@ project(":samza-autoscaling_$scalaVersion") { sourceSets.main.java.srcDirs = [] sourceSets.test.java.srcDirs = [] + jar { + manifest { + attributes("Implementation-Version": "$version") + } + } + dependencies { compile project(':samza-api') - compile project(":samza-core_$scalaVersion") - compile "org.scala-lang:scala-library:$scalaLibVersion" - compile "org.slf4j:slf4j-api:$slf4jVersion" + compile("com.101tec:zkclient:$zkClientVersion") { + exclude module: 'junit:junit' + } + compile "com.google.guava:guava:$guavaVersion" compile "net.sf.jopt-simple:jopt-simple:$joptSimpleVersion" + compile "org.apache.commons:commons-collections4:$apacheCommonsCollections4Version" + compile "org.apache.commons:commons-lang3:$commonsLang3Version" 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: 'slf4j-log4j12' - exclude module: 'servlet-api' - } - compile("org.apache.hadoop:hadoop-common:$yarnVersion") { - exclude module: 'slf4j-log4j12' - exclude module: 'servlet-api' - } - compile "org.apache.httpcomponents:httpclient:$httpClientVersion" + compile "org.scala-lang:scala-library:$scalaLibVersion" + compile "org.slf4j:slf4j-api:$slf4jVersion" + testCompile project(":samza-api").sourceSets.test.output testCompile "junit:junit:$junitVersion" testCompile "org.mockito:mockito-core:$mockitoVersion" + testCompile "org.powermock:powermock-api-mockito:$powerMockVersion" + testCompile "org.powermock:powermock-core:$powerMockVersion" + testCompile "org.powermock:powermock-module-junit4:$powerMockVersion" testCompile "org.scalatest:scalatest_$scalaVersion:$scalaTestVersion" } @@ -310,115 +309,54 @@ project(':samza-elasticsearch') { } } -project(':samza-sql') { - apply plugin: 'java' - - dependencies { - compile project(':samza-api') - compile project(":samza-kafka_$scalaVersion") - compile "org.apache.avro:avro:$avroVersion" - compile "org.apache.calcite:calcite-core:$calciteVersion" - compile "org.slf4j:slf4j-api:$slf4jVersion" - - testCompile project(":samza-test_$scalaVersion") - testCompile "junit:junit:$junitVersion" - testCompile "org.mockito:mockito-core:$mockitoVersion" - - testRuntime "org.slf4j:slf4j-simple:$slf4jVersion" - } -} - -project(':samza-tools') { - apply plugin: 'java' - - dependencies { - compile project(':samza-sql') - compile project(':samza-api') - compile project(':samza-azure') - compile "log4j:log4j:$log4jVersion" - compile "org.slf4j:slf4j-api:$slf4jVersion" - compile "org.slf4j:slf4j-log4j12:$slf4jVersion" - compile "commons-cli:commons-cli:$commonsCliVersion" - compile "org.apache.avro:avro:$avroVersion" - compile "org.apache.commons:commons-lang3:$commonsLang3Version" - compile "org.apache.kafka:kafka-clients:$kafkaVersion" - } - - tasks.create(name: "releaseToolsTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { - into "samza-tools-${version}" - compression = Compression.GZIP - from(project.file("./scripts")) { into "scripts/" } - from(project.file("./config")) { into "config/" } - from(project(':samza-shell').file("src/main/bash/run-class.sh")) { into "scripts/" } - from(configurations.runtime) { into("lib/") } - from(configurations.archives.artifacts.files) { into("lib/") } - duplicatesStrategy 'exclude' - } -} - -project(":samza-kafka_$scalaVersion") { +project(":samza-hdfs_$scalaVersion") { apply plugin: 'scala' // 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 = [] - - configurations { - // Remove transitive dependencies from Zookeeper that we don't want. - compile.exclude group: 'javax.jms', module: 'jms' - compile.exclude group: 'com.sun.jdmk', module: 'jmxtools' - compile.exclude group: 'com.sun.jmx', module: 'jmxri' - } dependencies { compile project(':samza-api') compile project(":samza-core_$scalaVersion") - compile "log4j:log4j:$log4jVersion" + // currently hdfs system producer/consumer do depend on yarn for two things: + // 1. staging directory 2. security + // SAMZA-1032 to solve the staging directory dependency + compile project(":samza-yarn_$scalaVersion") compile "org.scala-lang:scala-library:$scalaLibVersion" - compile "com.101tec:zkclient:$zkClientVersion" - compile "org.apache.zookeeper:zookeeper:$zookeeperVersion" - compile "org.codehaus.jackson:jackson-mapper-asl:$jacksonVersion" - compile "org.apache.kafka:kafka_$scalaVersion:$kafkaVersion" - compile "org.apache.kafka:kafka-clients:$kafkaVersion" - testCompile "org.apache.kafka:kafka_$scalaVersion:$kafkaVersion:test" - testCompile "org.apache.kafka:kafka-clients:$kafkaVersion:test" + compile("org.apache.hadoop:hadoop-hdfs:$yarnVersion") { + exclude module: 'slf4j-log4j12' + exclude module: 'servlet-api' + } + compile("org.apache.hadoop:hadoop-common:$yarnVersion") { + exclude module: 'slf4j-log4j12' + exclude module: 'servlet-api' + exclude module: 'zookeeper' + } + testCompile "junit:junit:$junitVersion" - testCompile "org.mockito:mockito-core:$mockitoVersion" testCompile "org.scalatest:scalatest_$scalaVersion:$scalaTestVersion" - testCompile project(":samza-core_$scalaVersion").sourceSets.test.output - - // Logging in tests is good. - testRuntime "org.slf4j:slf4j-simple:$slf4jVersion" - } - - test { - // Bump up the heap so we can start ZooKeeper and Kafka brokers. - minHeapSize = "1560m" - maxHeapSize = "1560m" - jvmArgs = ["-XX:+UseConcMarkSweepGC", "-server"] - // There appear to be issues between TestKafkaSystemAdmin and - // TestKafkaCheckpointManager both running brokeres and ZK. Restarting the - // gradle worker after every test clears things up. These tests should be - // moved to the integration test suite. - forkEvery = 1 + testCompile "org.apache.hadoop:hadoop-minicluster:$yarnVersion" } } -project(':samza-log4j') { - apply plugin: 'java' +project(":samza-inmemory") { apply plugin: 'checkstyle' + apply plugin: 'java' dependencies { - compile "log4j:log4j:$log4jVersion" + // internal dependencies compile project(':samza-api') - compile project(":samza-core_$scalaVersion") - compile "org.codehaus.jackson:jackson-mapper-asl:$jacksonVersion" + + // external libraries + compile "com.google.guava:guava:$guavaVersion" + compile "org.slf4j:slf4j-api:$slf4jVersion" + + + // test dependencies testCompile "junit:junit:$junitVersion" + testCompile "org.mockito:mockito-core:$mockitoVersion" } checkstyle { @@ -427,9 +365,8 @@ project(':samza-log4j') { } } -project(":samza-yarn_$scalaVersion") { +project(":samza-kafka_$scalaVersion") { apply plugin: 'scala' - apply plugin: 'lesscss' // Force scala joint compilation sourceSets.main.scala.srcDir "src/main/java" @@ -440,126 +377,43 @@ project(":samza-yarn_$scalaVersion") { sourceSets.main.java.srcDirs = [] sourceSets.test.java.srcDirs = [] - dependencies { - compile project(':samza-api') - compile project(":samza-core_$scalaVersion") - compile "org.scala-lang:scala-library:$scalaLibVersion" - compile "org.scala-lang:scala-compiler:$scalaLibVersion" - compile "org.codehaus.jackson:jackson-mapper-asl:$jacksonVersion" - compile "commons-httpclient:commons-httpclient:$commonsHttpClientVersion" - compile("org.apache.hadoop:hadoop-yarn-api:$yarnVersion") { - exclude module: 'slf4j-log4j12' - } - compile("org.apache.hadoop:hadoop-yarn-common:$yarnVersion") { - exclude module: 'slf4j-log4j12' - exclude module: 'servlet-api' - } - compile("org.apache.hadoop:hadoop-yarn-client:$yarnVersion") { - exclude module: 'slf4j-log4j12' - exclude module: 'servlet-api' - } - compile("org.apache.hadoop:hadoop-common:$yarnVersion") { - exclude module: 'slf4j-log4j12' - exclude module: 'servlet-api' - // Exclude because YARN's 3.4.5 ZK version is incompatbile with Kafka's 3.3.4. - exclude module: 'zookeeper' - } - compile("org.apache.hadoop:hadoop-hdfs:$yarnVersion") { - exclude module: 'slf4j-log4j12' - exclude module: 'servlet-api' - } - compile("org.scalatra:scalatra_$scalaVersion:$scalatraVersion") { - exclude module: 'scala-compiler' - exclude module: 'slf4j-api' - } - compile("org.scalatra:scalatra-scalate_$scalaVersion:$scalatraVersion") { - exclude module: 'scala-compiler' - exclude module: 'slf4j-api' - } - compile "joda-time:joda-time:$jodaTimeVersion" - compile "org.apache.zookeeper:zookeeper:$zookeeperVersion" - testCompile "junit:junit:$junitVersion" - testCompile "org.mockito:mockito-core:$mockitoVersion" - testCompile project(":samza-core_$scalaVersion").sourceSets.test.output - testCompile "org.scalatest:scalatest_$scalaVersion:$scalaTestVersion" - } - - repositories { - maven { - url "http://repo.typesafe.com/typesafe/releases" - } - } - - lesscss { - source = fileTree('src/main/less') { - include 'main.less' - } - dest = "$buildDir/resources/main/scalate/css" - } - - jar.dependsOn("lesscss") -} - -project(":samza-shell") { - apply plugin: 'java' - configurations { - gradleShell + // Remove transitive dependencies from Zookeeper that we don't want. + compile.exclude group: 'javax.jms', module: 'jms' + compile.exclude group: 'com.sun.jdmk', module: 'jmxtools' + compile.exclude group: 'com.sun.jmx', module: 'jmxri' } dependencies { - gradleShell project(":samza-core_$scalaVersion") - gradleShell project(":samza-kafka_$scalaVersion") - gradleShell project(":samza-test_$scalaVersion") - gradleShell project(":samza-yarn_$scalaVersion") - gradleShell "org.slf4j:slf4j-log4j12:$slf4jVersion" - gradleShell "log4j:log4j:1.2.16" - } - - task shellTarGz(type: Tar) { - compression = Compression.GZIP - classifier = 'dist' - from 'src/main/bash' - from 'src/main/resources' - from 'src/main/visualizer' - } - - artifacts { - archives(shellTarGz) { - name 'samza-shell' - classifier 'dist' - } - } - - // Usage: ./gradlew samza-shell:runJob \ - // -PconfigPath=file:///path/to/job/config.properties - task runJob(type:JavaExec) { - description 'To run a job (defined in a properties file)' - main = 'org.apache.samza.job.JobRunner' - classpath = configurations.gradleShell - if (project.hasProperty('configPath')) args += ['--config-path', configPath] - jvmArgs = ["-Dlog4j.configuration=file:src/main/resources/log4j-console.xml"] - } + compile project(':samza-api') + compile project(":samza-core_$scalaVersion") + compile "org.scala-lang:scala-library:$scalaLibVersion" + compile "com.101tec:zkclient:$zkClientVersion" + compile "org.apache.zookeeper:zookeeper:$zookeeperVersion" + compile "org.codehaus.jackson:jackson-mapper-asl:$jacksonVersion" + compile "org.apache.kafka:kafka_$scalaVersion:$kafkaVersion" + compile "org.apache.kafka:kafka-clients:$kafkaVersion" + testCompile "org.apache.kafka:kafka_$scalaVersion:$kafkaVersion:test" + testCompile "org.apache.kafka:kafka-clients:$kafkaVersion:test" + testCompile "junit:junit:$junitVersion" + testCompile "org.mockito:mockito-core:$mockitoVersion" + testCompile "org.scalatest:scalatest_$scalaVersion:$scalaTestVersion" + testCompile project(":samza-core_$scalaVersion").sourceSets.test.output - // Usage: ./gradlew samza-shell:checkpointTool \ - // -PconfigPath=file:///path/to/job/config.properties -PnewOffsets=file:///path/to/new/offsets.properties - task checkpointTool(type:JavaExec) { - description 'Command-line tool to inspect and manipulate the job’s checkpoint' - main = 'org.apache.samza.checkpoint.CheckpointTool' - classpath = configurations.gradleShell - if (project.hasProperty('configPath')) args += ['--config-path', configPath] - if (project.hasProperty('newOffsets')) args += ['--new-offsets', newOffsets] - jvmArgs = ["-Dlog4j.configuration=file:src/main/resources/log4j-console.xml"] + // Logging in tests is good. + testRuntime "org.slf4j:slf4j-simple:$slf4jVersion" } - // Usage: ./gradlew samza-shell:kvPerformanceTest - // -PconfigPath=file:///path/to/job/config.properties - task kvPerformanceTest(type:JavaExec) { - description 'Command-line tool to run key-value performance tests' - main = 'org.apache.samza.test.performance.TestKeyValuePerformance' - classpath = configurations.gradleShell - if (project.hasProperty('configPath')) args += ['--config-path', configPath] - jvmArgs = ["-Dlog4j.configuration=file:src/main/resources/log4j-console.xml"] + test { + // Bump up the heap so we can start ZooKeeper and Kafka brokers. + minHeapSize = "1560m" + maxHeapSize = "1560m" + jvmArgs = ["-XX:+UseConcMarkSweepGC", "-server"] + // There appear to be issues between TestKafkaSystemAdmin and + // TestKafkaCheckpointManager both running brokeres and ZK. Restarting the + // gradle worker after every test clears things up. These tests should be + // moved to the integration test suite. + forkEvery = 1 } } @@ -636,35 +490,21 @@ project(":samza-kv-rocksdb_$scalaVersion") { } } -project(":samza-hdfs_$scalaVersion") { - apply plugin: 'scala' - - // Force scala joint compilation - sourceSets.main.scala.srcDir "src/main/java" - sourceSets.test.scala.srcDir "src/test/java" - sourceSets.main.java.srcDirs = [] +project(':samza-log4j') { + apply plugin: 'java' + apply plugin: 'checkstyle' dependencies { + compile "log4j:log4j:$log4jVersion" compile project(':samza-api') compile project(":samza-core_$scalaVersion") - // currently hdfs system producer/consumer do depend on yarn for two things: - // 1. staging directory 2. security - // SAMZA-1032 to solve the staging directory dependency - compile project(":samza-yarn_$scalaVersion") - compile "org.scala-lang:scala-library:$scalaLibVersion" - compile("org.apache.hadoop:hadoop-hdfs:$yarnVersion") { - exclude module: 'slf4j-log4j12' - exclude module: 'servlet-api' - } - compile("org.apache.hadoop:hadoop-common:$yarnVersion") { - exclude module: 'slf4j-log4j12' - exclude module: 'servlet-api' - exclude module: 'zookeeper' - } - + compile "org.codehaus.jackson:jackson-mapper-asl:$jacksonVersion" testCompile "junit:junit:$junitVersion" - testCompile "org.scalatest:scalatest_$scalaVersion:$scalaTestVersion" - testCompile "org.apache.hadoop:hadoop-minicluster:$yarnVersion" + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + toolVersion = "$checkstyleVersion" } } @@ -728,6 +568,87 @@ project(":samza-rest") { } } +project(":samza-shell") { + apply plugin: 'java' + + configurations { + gradleShell + } + + dependencies { + gradleShell project(":samza-core_$scalaVersion") + gradleShell project(":samza-kafka_$scalaVersion") + gradleShell project(":samza-test_$scalaVersion") + gradleShell project(":samza-yarn_$scalaVersion") + gradleShell "org.slf4j:slf4j-log4j12:$slf4jVersion" + gradleShell "log4j:log4j:1.2.16" + } + + task shellTarGz(type: Tar) { + compression = Compression.GZIP + classifier = 'dist' + from 'src/main/bash' + from 'src/main/resources' + from 'src/main/visualizer' + } + + artifacts { + archives(shellTarGz) { + name 'samza-shell' + classifier 'dist' + } + } + + // Usage: ./gradlew samza-shell:runJob \ + // -PconfigPath=file:///path/to/job/config.properties + task runJob(type:JavaExec) { + description 'To run a job (defined in a properties file)' + main = 'org.apache.samza.job.JobRunner' + classpath = configurations.gradleShell + if (project.hasProperty('configPath')) args += ['--config-path', configPath] + jvmArgs = ["-Dlog4j.configuration=file:src/main/resources/log4j-console.xml"] + } + + // Usage: ./gradlew samza-shell:checkpointTool \ + // -PconfigPath=file:///path/to/job/config.properties -PnewOffsets=file:///path/to/new/offsets.properties + task checkpointTool(type:JavaExec) { + description 'Command-line tool to inspect and manipulate the job’s checkpoint' + main = 'org.apache.samza.checkpoint.CheckpointTool' + classpath = configurations.gradleShell + if (project.hasProperty('configPath')) args += ['--config-path', configPath] + if (project.hasProperty('newOffsets')) args += ['--new-offsets', newOffsets] + jvmArgs = ["-Dlog4j.configuration=file:src/main/resources/log4j-console.xml"] + } + + // Usage: ./gradlew samza-shell:kvPerformanceTest + // -PconfigPath=file:///path/to/job/config.properties + task kvPerformanceTest(type:JavaExec) { + description 'Command-line tool to run key-value performance tests' + main = 'org.apache.samza.test.performance.TestKeyValuePerformance' + classpath = configurations.gradleShell + if (project.hasProperty('configPath')) args += ['--config-path', configPath] + jvmArgs = ["-Dlog4j.configuration=file:src/main/resources/log4j-console.xml"] + } +} + +project(':samza-sql') { + apply plugin: 'java' + + dependencies { + compile project(':samza-api') + compile project(":samza-kafka_$scalaVersion") + compile "org.apache.avro:avro:$avroVersion" + compile "org.apache.calcite:calcite-core:$calciteVersion" + compile "org.slf4j:slf4j-api:$slf4jVersion" + + testCompile project(":samza-test_$scalaVersion") + testCompile "junit:junit:$junitVersion" + testCompile "org.mockito:mockito-core:$mockitoVersion" + + testRuntime "org.slf4j:slf4j-simple:$slf4jVersion" + } +} + project(":samza-test_$scalaVersion") { apply plugin: 'scala' apply plugin: 'checkstyle' @@ -753,6 +674,7 @@ project(":samza-test_$scalaVersion") { compile project(":samza-kv-inmemory_$scalaVersion") compile project(":samza-kv-rocksdb_$scalaVersion") compile project(":samza-core_$scalaVersion") + compile project(":samza-inmemory") runtime project(":samza-log4j") runtime project(":samza-yarn_$scalaVersion") runtime project(":samza-kafka_$scalaVersion") @@ -761,16 +683,16 @@ project(":samza-test_$scalaVersion") { compile "net.sf.jopt-simple:jopt-simple:$joptSimpleVersion" compile "javax.mail:mail:1.4" compile "org.apache.kafka:kafka_$scalaVersion:$kafkaVersion" + compile "junit:junit:$junitVersion" + compile "org.hamcrest:hamcrest-all:$hamcrestVersion" testCompile "org.apache.kafka:kafka_$scalaVersion:$kafkaVersion:test" testCompile "com.101tec:zkclient:$zkClientVersion" - testCompile "junit:junit:$junitVersion" testCompile project(":samza-kafka_$scalaVersion") testCompile "org.apache.kafka:kafka_$scalaVersion:$kafkaVersion:test" testCompile "org.apache.kafka:kafka-clients:$kafkaVersion:test" testCompile project(":samza-core_$scalaVersion").sourceSets.test.output testCompile "org.scalatest:scalatest_$scalaVersion:$scalaTestVersion" testCompile "org.mockito:mockito-core:$mockitoVersion" - testCompile "org.hamcrest:hamcrest-all:$hamcrestVersion" testRuntime "org.slf4j:slf4j-simple:$slf4jVersion" } @@ -801,3 +723,104 @@ project(":samza-test_$scalaVersion") { from(configurations.archives.artifacts.files) { into("lib/") } } } + +project(':samza-tools') { + apply plugin: 'java' + + dependencies { + compile project(':samza-sql') + compile project(':samza-api') + compile project(':samza-azure') + compile "log4j:log4j:$log4jVersion" + compile "org.slf4j:slf4j-api:$slf4jVersion" + compile "org.slf4j:slf4j-log4j12:$slf4jVersion" + compile "commons-cli:commons-cli:$commonsCliVersion" + compile "org.apache.avro:avro:$avroVersion" + compile "org.apache.commons:commons-lang3:$commonsLang3Version" + compile "org.apache.kafka:kafka-clients:$kafkaVersion" + } + + tasks.create(name: "releaseToolsTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { + into "samza-tools-${version}" + compression = Compression.GZIP + from(project.file("./scripts")) { into "scripts/" } + from(project.file("./config")) { into "config/" } + from(project(':samza-shell').file("src/main/bash/run-class.sh")) { into "scripts/" } + from(configurations.runtime) { into("lib/") } + from(configurations.archives.artifacts.files) { into("lib/") } + duplicatesStrategy 'exclude' + } +} + +project(":samza-yarn_$scalaVersion") { + apply plugin: 'scala' + apply plugin: 'lesscss' + + // 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_$scalaVersion") + compile "org.scala-lang:scala-library:$scalaLibVersion" + compile "org.scala-lang:scala-compiler:$scalaLibVersion" + compile "org.codehaus.jackson:jackson-mapper-asl:$jacksonVersion" + compile "commons-httpclient:commons-httpclient:$commonsHttpClientVersion" + compile("org.apache.hadoop:hadoop-yarn-api:$yarnVersion") { + exclude module: 'slf4j-log4j12' + } + compile("org.apache.hadoop:hadoop-yarn-common:$yarnVersion") { + exclude module: 'slf4j-log4j12' + exclude module: 'servlet-api' + } + compile("org.apache.hadoop:hadoop-yarn-client:$yarnVersion") { + exclude module: 'slf4j-log4j12' + exclude module: 'servlet-api' + } + compile("org.apache.hadoop:hadoop-common:$yarnVersion") { + exclude module: 'slf4j-log4j12' + exclude module: 'servlet-api' + // Exclude because YARN's 3.4.5 ZK version is incompatbile with Kafka's 3.3.4. + exclude module: 'zookeeper' + } + compile("org.apache.hadoop:hadoop-hdfs:$yarnVersion") { + exclude module: 'slf4j-log4j12' + exclude module: 'servlet-api' + } + compile("org.scalatra:scalatra_$scalaVersion:$scalatraVersion") { + exclude module: 'scala-compiler' + exclude module: 'slf4j-api' + } + compile("org.scalatra:scalatra-scalate_$scalaVersion:$scalatraVersion") { + exclude module: 'scala-compiler' + exclude module: 'slf4j-api' + } + compile "joda-time:joda-time:$jodaTimeVersion" + compile "org.apache.zookeeper:zookeeper:$zookeeperVersion" + testCompile "junit:junit:$junitVersion" + testCompile "org.mockito:mockito-core:$mockitoVersion" + testCompile project(":samza-core_$scalaVersion").sourceSets.test.output + testCompile "org.scalatest:scalatest_$scalaVersion:$scalaTestVersion" + } + + repositories { + maven { + url "http://repo.typesafe.com/typesafe/releases" + } + } + + lesscss { + source = fileTree('src/main/less') { + include 'main.less' + } + dest = "$buildDir/resources/main/scalate/css" + } + + jar.dependsOn("lesscss") +} diff --git a/samza-api/src/main/java/org/apache/samza/application/StreamApplication.java b/samza-api/src/main/java/org/apache/samza/application/StreamApplication.java index f615207f62..a6d3204bf4 100644 --- a/samza-api/src/main/java/org/apache/samza/application/StreamApplication.java +++ b/samza-api/src/main/java/org/apache/samza/application/StreamApplication.java @@ -72,7 +72,7 @@ * See {@link InitableFunction} and {@link org.apache.samza.operators.functions.ClosableFunction}. */ @InterfaceStability.Unstable -public interface StreamApplication { +public interface StreamApplication { /** * Describes and initializes the transforms for processing message streams and generating results. diff --git a/samza-api/src/main/java/org/apache/samza/operators/TimerRegistry.java b/samza-api/src/main/java/org/apache/samza/operators/TimerRegistry.java new file mode 100644 index 0000000000..64dd4ec96d --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/operators/TimerRegistry.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.operators; + +/** + * Allows registering epoch-time timer callbacks from the operators. + * See {@link org.apache.samza.operators.functions.TimerFunction} for details. + * @param type of the timer key + */ +public interface TimerRegistry { + + /** + * Register a epoch-time timer with key. + * @param key unique timer key + * @param timestamp epoch time when the timer will be fired, in milliseconds + */ + void register(K key, long timestamp); + + /** + * Delete the timer for the provided key. + * @param key key for the timer to delete + */ + void delete(K key); +} diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java index b08c6cdc9e..6651819780 100644 --- a/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/InitableFunction.java @@ -16,13 +16,13 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.samza.operators.functions; import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.config.Config; import org.apache.samza.task.TaskContext; - /** * A function that can be initialized before execution. * @@ -41,5 +41,4 @@ public interface InitableFunction { * @param context the {@link TaskContext} for this task */ default void init(Config config, TaskContext context) { } - } diff --git a/samza-api/src/main/java/org/apache/samza/operators/functions/TimerFunction.java b/samza-api/src/main/java/org/apache/samza/operators/functions/TimerFunction.java new file mode 100644 index 0000000000..01825c6479 --- /dev/null +++ b/samza-api/src/main/java/org/apache/samza/operators/functions/TimerFunction.java @@ -0,0 +1,65 @@ +/* + * 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.operators.functions; + +import org.apache.samza.operators.TimerRegistry; + +import java.util.Collection; + +/** + * Allows timer registration with a key and is invoked when the timer is fired. + * Key must be a unique identifier for this timer, and is provided in the callback when the timer fires. + * + *

+ * Example of a {@link FlatMapFunction} with timer: + *

{@code
+ *    public class ExampleTimerFn implements FlatMapFunction, TimerFunction {
+ *      public void registerTimer(TimerRegistry timerRegistry) {
+ *        long time = System.currentTimeMillis() + 5000; // fire after 5 sec
+ *        timerRegistry.register("example-timer", time);
+ *      }
+ *      public Collection apply(String s) {
+ *        ...
+ *      }
+ *      public Collection onTimer(String key, long timestamp) {
+ *        // example-timer fired
+ *        ...
+ *      }
+ *    }
+ * }
+ * @param type of the key + * @param type of the output + */ +public interface TimerFunction { + + /** + * Registers any epoch-time timers using the registry + * @param timerRegistry a keyed {@link TimerRegistry} + */ + void registerTimer(TimerRegistry timerRegistry); + + /** + * Returns the output after the timer with key fires. + * @param key timer key + * @param timestamp time of the epoch-time timer fired, in milliseconds + * @return {@link Collection} of output elements + */ + Collection onTimer(K key, long timestamp); +} diff --git a/samza-api/src/main/java/org/apache/samza/table/ReadableTable.java b/samza-api/src/main/java/org/apache/samza/table/ReadableTable.java index 5ad6e0f627..15b6115fb1 100644 --- a/samza-api/src/main/java/org/apache/samza/table/ReadableTable.java +++ b/samza-api/src/main/java/org/apache/samza/table/ReadableTable.java @@ -22,7 +22,9 @@ import java.util.Map; import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.container.SamzaContainerContext; import org.apache.samza.operators.KV; +import org.apache.samza.task.TaskContext; /** @@ -34,6 +36,14 @@ */ @InterfaceStability.Unstable public interface ReadableTable extends Table> { + /** + * Initializes the table during container initialization. + * Guaranteed to be invoked as the first operation on the table. + * @param containerContext Samza container context + * @param taskContext nullable for global table + */ + default void init(SamzaContainerContext containerContext, TaskContext taskContext) { + } /** * Gets the value associated with the specified {@code key}. @@ -57,5 +67,4 @@ public interface ReadableTable extends Table> { * Close the table and release any resources acquired */ void close(); - } diff --git a/samza-api/src/main/java/org/apache/samza/table/TableProvider.java b/samza-api/src/main/java/org/apache/samza/table/TableProvider.java index 54c6f5d088..bbbe38a731 100644 --- a/samza-api/src/main/java/org/apache/samza/table/TableProvider.java +++ b/samza-api/src/main/java/org/apache/samza/table/TableProvider.java @@ -21,6 +21,8 @@ import java.util.Map; import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.container.SamzaContainerContext; +import org.apache.samza.task.TaskContext; /** @@ -29,6 +31,13 @@ */ @InterfaceStability.Unstable public interface TableProvider { + /** + * Initialize TableProvider with container and task context + * @param containerContext Samza container context + * @param taskContext nullable for global table + */ + void init(SamzaContainerContext containerContext, TaskContext taskContext); + /** * Get an instance of the table for read/write operations * @return the underlying table @@ -46,12 +55,7 @@ public interface TableProvider { Map generateConfig(Map config); /** - * Start the underlying table - */ - void start(); - - /** - * Stop the underlying table + * Shutdown the underlying table */ - void stop(); + void close(); } diff --git a/samza-api/src/main/java/org/apache/samza/task/TaskContext.java b/samza-api/src/main/java/org/apache/samza/task/TaskContext.java index 11ffacc35f..ea2a3bca98 100644 --- a/samza-api/src/main/java/org/apache/samza/task/TaskContext.java +++ b/samza-api/src/main/java/org/apache/samza/task/TaskContext.java @@ -74,4 +74,23 @@ default void setUserContext(Object context) { } default Object getUserContext() { return null; } + + /** + * Register a keyed timer with a callback of {@link TimerCallback} in this task. + * The callback will be invoked exclusively with any other operations for this task, + * e.g. processing, windowing and commit. + * @param key timer key + * @param timestamp epoch time when the timer will be fired, in milliseconds + * @param callback callback when the timer is fired + * @param type of the key + */ + void registerTimer(K key, long timestamp, TimerCallback callback); + + /** + * Delete the keyed timer in this task. + * Deletion only happens if the timer hasn't been fired. Otherwise it will not interrupt. + * @param key timer key + * @param type of the key + */ + void deleteTimer(K key); } diff --git a/samza-api/src/main/java/org/apache/samza/table/LocalStoreBackedTableProvider.java b/samza-api/src/main/java/org/apache/samza/task/TimerCallback.java similarity index 59% rename from samza-api/src/main/java/org/apache/samza/table/LocalStoreBackedTableProvider.java rename to samza-api/src/main/java/org/apache/samza/task/TimerCallback.java index 21630ab10f..3add129c63 100644 --- a/samza-api/src/main/java/org/apache/samza/table/LocalStoreBackedTableProvider.java +++ b/samza-api/src/main/java/org/apache/samza/task/TimerCallback.java @@ -16,22 +16,19 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.samza.table; - -import org.apache.samza.storage.StorageEngine; +package org.apache.samza.task; /** - * Interface for tables backed by Samza local stores. The backing stores are - * injected during initialization of the table. Since the lifecycle - * of the underlying stores are already managed by Samza container, - * the table provider will not manage the lifecycle of the backing - * stores. + * The callback that is invoked when its corresponding timer registered via {@link TaskContext} fires. + * @param type of the timer key */ -public interface LocalStoreBackedTableProvider extends TableProvider { +public interface TimerCallback { /** - * Initializes the table provider with the backing store - * @param store the backing store + * Invoked when the timer of key fires. + * @param key timer key + * @param collector contains the means of sending message envelopes to the output stream. + * @param coordinator manages execution of tasks. */ - void init(StorageEngine store); + void onTimer(K key, MessageCollector collector, TaskCoordinator coordinator); } diff --git a/samza-api/src/main/java/org/apache/samza/util/RateLimiter.java b/samza-api/src/main/java/org/apache/samza/util/RateLimiter.java index 75818ddd74..ad40d35420 100644 --- a/samza-api/src/main/java/org/apache/samza/util/RateLimiter.java +++ b/samza-api/src/main/java/org/apache/samza/util/RateLimiter.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; import org.apache.samza.annotation.InterfaceStability; @@ -43,7 +44,6 @@ *
    *
  • Block indefinitely until requested credits become available
  • *
  • Block for a provided amount of time, then return available credits
  • - *
  • Non-blocking, returns immediately available credits
  • *
* */ @@ -79,15 +79,6 @@ public interface RateLimiter extends Serializable { */ int acquire(int numberOfCredit, long timeout, TimeUnit unit); - /** - * Attempt to acquire the provided number of credits, returns immediately number of - * credits acquired. - * - * @param numberOfCredit requested number of credits - * @return number of credits acquired - */ - int tryAcquire(int numberOfCredit); - /** * Attempt to acquire the provided number of credits for a number of tags, blocks indefinitely * until all requested credits become available @@ -110,11 +101,8 @@ public interface RateLimiter extends Serializable { Map acquire(Map tagToCreditMap, long timeout, TimeUnit unit); /** - * Attempt to acquire the provided number of credits for a number of tags, returns immediately number of - * credits acquired. - * - * @param tagToCreditMap a map of requested number of credits keyed by tag - * @return a map of number of credits acquired keyed by tag + * Get the entire set of tags for which we have configured credits for rate limiting. + * @return set of supported tags */ - Map tryAcquire(Map tagToCreditMap); + Set getSupportedTags(); } diff --git a/samza-core/src/main/java/org/apache/samza/container/TaskContextImpl.java b/samza-core/src/main/java/org/apache/samza/container/TaskContextImpl.java index 0248486431..0d76a33903 100644 --- a/samza-core/src/main/java/org/apache/samza/container/TaskContextImpl.java +++ b/samza-core/src/main/java/org/apache/samza/container/TaskContextImpl.java @@ -19,10 +19,7 @@ package org.apache.samza.container; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - +import com.google.common.collect.ImmutableSet; import org.apache.samza.checkpoint.OffsetManager; import org.apache.samza.job.model.JobModel; import org.apache.samza.metrics.ReadableMetricsRegistry; @@ -32,11 +29,16 @@ import org.apache.samza.system.SystemStreamPartition; import org.apache.samza.table.Table; import org.apache.samza.table.TableManager; +import org.apache.samza.task.SystemTimerScheduler; import org.apache.samza.task.TaskContext; +import org.apache.samza.task.TimerCallback; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.collect.ImmutableSet; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; public class TaskContextImpl implements TaskContext { private static final Logger LOG = LoggerFactory.getLogger(TaskContextImpl.class); @@ -51,6 +53,7 @@ public class TaskContextImpl implements TaskContext { private final JobModel jobModel; private final StreamMetadataCache streamMetadataCache; private final Map objectRegistry = new HashMap<>(); + private final SystemTimerScheduler timerScheduler; private Object userContext = null; @@ -62,7 +65,8 @@ public TaskContextImpl(TaskName taskName, TaskStorageManager storageManager, TableManager tableManager, JobModel jobModel, - StreamMetadataCache streamMetadataCache) { + StreamMetadataCache streamMetadataCache, + ScheduledExecutorService timerExecutor) { this.taskName = taskName; this.metrics = metrics; this.containerContext = containerContext; @@ -72,6 +76,7 @@ public TaskContextImpl(TaskName taskName, this.tableManager = tableManager; this.jobModel = jobModel; this.streamMetadataCache = streamMetadataCache; + this.timerScheduler = SystemTimerScheduler.create(timerExecutor); } @Override @@ -129,6 +134,16 @@ public Object getUserContext() { return userContext; } + @Override + public void registerTimer(K key, long timestamp, TimerCallback callback) { + timerScheduler.setTimer(key, timestamp, callback); + } + + @Override + public void deleteTimer(K key) { + timerScheduler.deleteTimer(key); + } + public void registerObject(String name, Object value) { objectRegistry.put(name, value); } @@ -144,4 +159,8 @@ public JobModel getJobModel() { public StreamMetadataCache getStreamMetadataCache() { return streamMetadataCache; } + + public SystemTimerScheduler getTimerScheduler() { + return timerScheduler; + } } diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java index 15b763de86..7219180627 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImpl.java @@ -25,6 +25,8 @@ import org.apache.samza.container.TaskName; import org.apache.samza.job.model.ContainerModel; import org.apache.samza.job.model.TaskModel; +import org.apache.samza.operators.TimerRegistry; +import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; import org.apache.samza.system.EndOfStreamMessage; import org.apache.samza.metrics.Counter; @@ -80,6 +82,7 @@ public abstract class OperatorImpl { private EndOfStreamStates eosStates; // watermark states private WatermarkStates watermarkStates; + private TaskContext taskContext; /** * Initialize this {@link OperatorImpl} and its user-defined functions. @@ -121,7 +124,8 @@ public final void init(Config config, TaskContext context) { this.usedInCurrentTask = true; } - handleInit(config, context); + this.taskContext = taskContext; + handleInit(config, taskContext); initialized = true; } @@ -415,6 +419,42 @@ final long getOutputWatermark() { } } + /** + * Returns a registry which allows registering arbitrary system-clock timer with K-typed key. + * The user-defined function in the operator spec needs to implement {@link TimerFunction#onTimer(Object, long)} + * for timer notifications. + * @param key type for the timer. + * @return an instance of {@link TimerRegistry} + */ + TimerRegistry createOperatorTimerRegistry() { + return new TimerRegistry() { + @Override + public void register(K key, long time) { + taskContext.registerTimer(key, time, (k, collector, coordinator) -> { + final TimerFunction timerFn = getOperatorSpec().getTimerFn(); + if (timerFn != null) { + final Collection output = timerFn.onTimer(key, time); + + if (!output.isEmpty()) { + output.forEach(rm -> + registeredOperators.forEach(op -> + op.onMessage(rm, collector, coordinator))); + } + } else { + throw new SamzaException( + String.format("Operator %s id %s (created at %s) must implement TimerFunction to use system timer.", + getOperatorSpec().getOpCode().name(), getOpImplId(), getOperatorSpec().getSourceLocation())); + } + }); + } + + @Override + public void delete(K key) { + taskContext.deleteTimer(key); + } + }; + } + public void close() { if (closed) { throw new IllegalStateException( diff --git a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java index 3882544e99..bbc878364b 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java +++ b/samza-core/src/main/java/org/apache/samza/operators/impl/OperatorImplGraph.java @@ -18,20 +18,15 @@ */ package org.apache.samza.operators.impl; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Lists; +import com.google.common.collect.Multimap; import org.apache.samza.config.Config; import org.apache.samza.container.TaskContextImpl; import org.apache.samza.job.model.JobModel; import org.apache.samza.operators.KV; import org.apache.samza.operators.StreamGraphImpl; +import org.apache.samza.operators.TimerRegistry; import org.apache.samza.operators.functions.JoinFunction; import org.apache.samza.operators.functions.PartialJoinFunction; import org.apache.samza.operators.impl.store.TimestampedValue; @@ -41,11 +36,11 @@ import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.operators.spec.OutputOperatorSpec; import org.apache.samza.operators.spec.PartitionByOperatorSpec; +import org.apache.samza.operators.spec.SendToTableOperatorSpec; import org.apache.samza.operators.spec.SinkOperatorSpec; import org.apache.samza.operators.spec.StreamOperatorSpec; import org.apache.samza.operators.spec.StreamTableJoinOperatorSpec; import org.apache.samza.operators.spec.WindowOperatorSpec; -import org.apache.samza.operators.spec.SendToTableOperatorSpec; import org.apache.samza.storage.kv.KeyValueStore; import org.apache.samza.system.SystemStream; import org.apache.samza.task.TaskContext; @@ -53,9 +48,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.Lists; -import com.google.common.collect.Multimap; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; /** * The DAG of {@link OperatorImpl}s corresponding to the DAG of {@link OperatorSpec}s. @@ -166,6 +166,11 @@ OperatorImpl createAndRegisterOperatorImpl(OperatorSpec prevOperatorSpec, Operat operatorImpl.init(config, context); operatorImpl.registerInputStream(inputStream); + if (operatorSpec.getTimerFn() != null) { + final TimerRegistry timerRegistry = operatorImpl.createOperatorTimerRegistry(); + operatorSpec.getTimerFn().registerTimer(timerRegistry); + } + // Note: The key here is opImplId, which may not equal opId for some impls (e.g. PartialJoinOperatorImpl). // This is currently OK since we don't need to look up a partial join operator impl again during traversal // (a join cannot have a cycle). diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/BroadcastOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/BroadcastOperatorSpec.java index 6689690f66..2c76e600a2 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/BroadcastOperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/BroadcastOperatorSpec.java @@ -20,6 +20,7 @@ package org.apache.samza.operators.spec; +import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; public class BroadcastOperatorSpec extends OperatorSpec { @@ -40,4 +41,9 @@ public OutputStreamImpl getOutputStream() { public WatermarkFunction getWatermarkFn() { return null; } + + @Override + public TimerFunction getTimerFn() { + return null; + } } diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java index 2ad0597c0f..2ed1e3040d 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/InputOperatorSpec.java @@ -19,6 +19,7 @@ package org.apache.samza.operators.spec; import org.apache.samza.operators.KV; +import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.serializers.Serde; import org.apache.samza.operators.functions.WatermarkFunction; import org.apache.samza.system.StreamSpec; @@ -65,5 +66,10 @@ public boolean isKeyed() { @Override public WatermarkFunction getWatermarkFn() { return null; - } + } + + @Override + public TimerFunction getTimerFn() { + return null; + } } diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java index c730bca62b..9e058ff424 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/JoinOperatorSpec.java @@ -20,6 +20,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.samza.operators.functions.JoinFunction; +import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; import org.apache.samza.operators.impl.store.TimestampedValueSerde; import org.apache.samza.operators.impl.store.TimestampedValue; @@ -97,6 +98,11 @@ public WatermarkFunction getWatermarkFn() { return joinFn instanceof WatermarkFunction ? (WatermarkFunction) joinFn : null; } + @Override + public TimerFunction getTimerFn() { + return joinFn instanceof TimerFunction ? (TimerFunction) joinFn : null; + } + public OperatorSpec getLeftInputOpSpec() { return leftInputOpSpec; } diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java index 00b5318bbb..7b0a41b394 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpec.java @@ -25,6 +25,7 @@ import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.MessageStreamImpl; +import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; /** @@ -130,4 +131,6 @@ public final String getSourceLocation() { } abstract public WatermarkFunction getWatermarkFn(); + + abstract public TimerFunction getTimerFn(); } diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java index 2a2e33a7ef..c38f6e8a5f 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OperatorSpecs.java @@ -95,7 +95,7 @@ public void init(Config config, TaskContext context) { public void close() { mapFn.close(); } - }, OperatorSpec.OpCode.MAP, opId); + }, mapFn, OperatorSpec.OpCode.MAP, opId); } /** @@ -129,7 +129,7 @@ public void init(Config config, TaskContext context) { public void close() { filterFn.close(); } - }, OperatorSpec.OpCode.FILTER, opId); + }, filterFn, OperatorSpec.OpCode.FILTER, opId); } /** @@ -143,7 +143,7 @@ public void close() { */ public static StreamOperatorSpec createFlatMapOperatorSpec( FlatMapFunction flatMapFn, String opId) { - return new StreamOperatorSpec<>((FlatMapFunction) flatMapFn, OperatorSpec.OpCode.FLAT_MAP, opId); + return new StreamOperatorSpec<>((FlatMapFunction) flatMapFn, flatMapFn, OperatorSpec.OpCode.FLAT_MAP, opId); } /** @@ -242,7 +242,7 @@ public static StreamOperatorSpec createMergeOperatorSpec(String opId) this.add(message); } }, - OperatorSpec.OpCode.MERGE, opId); + null, OperatorSpec.OpCode.MERGE, opId); } /** diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/OutputOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/OutputOperatorSpec.java index 6cb4fca93b..40a5c0ec06 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/OutputOperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/OutputOperatorSpec.java @@ -19,6 +19,7 @@ package org.apache.samza.operators.spec; +import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; /** @@ -56,4 +57,9 @@ public OutputStreamImpl getOutputStream() { public WatermarkFunction getWatermarkFn() { return null; } + + @Override + public TimerFunction getTimerFn() { + return null; + } } diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/PartitionByOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/PartitionByOperatorSpec.java index 399c836d03..a0a9b6167a 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/PartitionByOperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/PartitionByOperatorSpec.java @@ -19,6 +19,7 @@ package org.apache.samza.operators.spec; import org.apache.samza.operators.KV; +import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; import java.util.function.Function; @@ -78,4 +79,9 @@ public OutputStreamImpl> getOutputStream() { public WatermarkFunction getWatermarkFn() { return null; } + + @Override + public TimerFunction getTimerFn() { + return null; + } } diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/SendToTableOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/SendToTableOperatorSpec.java index 9084be2fad..e1b51bea6a 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/SendToTableOperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/SendToTableOperatorSpec.java @@ -20,6 +20,7 @@ import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.operators.KV; +import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; import org.apache.samza.table.TableSpec; @@ -62,4 +63,9 @@ public TableSpec getTableSpec() { public WatermarkFunction getWatermarkFn() { return null; } + + @Override + public TimerFunction getTimerFn() { + return null; + } } diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java index 1ca380112d..aa0f066947 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/SinkOperatorSpec.java @@ -19,6 +19,7 @@ package org.apache.samza.operators.spec; import org.apache.samza.operators.functions.SinkFunction; +import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; @@ -54,4 +55,9 @@ public SinkFunction getSinkFn() { public WatermarkFunction getWatermarkFn() { return sinkFn instanceof WatermarkFunction ? (WatermarkFunction) sinkFn : null; } + + @Override + public TimerFunction getTimerFn() { + return sinkFn instanceof TimerFunction ? (TimerFunction) sinkFn : null; + } } diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java index b1e29c6562..644eb6ca45 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/StreamOperatorSpec.java @@ -19,6 +19,7 @@ package org.apache.samza.operators.spec; import org.apache.samza.operators.functions.FlatMapFunction; +import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; @@ -31,17 +32,20 @@ public class StreamOperatorSpec extends OperatorSpec { private final FlatMapFunction transformFn; + private final Object originalFn; /** * Constructor for a {@link StreamOperatorSpec}. * * @param transformFn the transformation function + * @param originalFn the original user function before wrapping to transformFn * @param opCode the {@link OpCode} for this {@link StreamOperatorSpec} * @param opId the unique ID for this {@link StreamOperatorSpec} */ - StreamOperatorSpec(FlatMapFunction transformFn, OperatorSpec.OpCode opCode, String opId) { + StreamOperatorSpec(FlatMapFunction transformFn, Object originalFn, OperatorSpec.OpCode opCode, String opId) { super(opCode, opId); this.transformFn = transformFn; + this.originalFn = originalFn; } public FlatMapFunction getTransformFn() { @@ -50,6 +54,11 @@ public FlatMapFunction getTransformFn() { @Override public WatermarkFunction getWatermarkFn() { - return transformFn instanceof WatermarkFunction ? (WatermarkFunction) transformFn : null; + return originalFn instanceof WatermarkFunction ? (WatermarkFunction) originalFn : null; + } + + @Override + public TimerFunction getTimerFn() { + return originalFn instanceof TimerFunction ? (TimerFunction) originalFn : null; } } diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/StreamTableJoinOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/StreamTableJoinOperatorSpec.java index 730913a3de..c7735c6d77 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/StreamTableJoinOperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/StreamTableJoinOperatorSpec.java @@ -20,6 +20,7 @@ import org.apache.samza.annotation.InterfaceStability; import org.apache.samza.operators.functions.StreamTableJoinFunction; +import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; import org.apache.samza.table.TableSpec; @@ -64,4 +65,9 @@ public WatermarkFunction getWatermarkFn() { return joinFn instanceof WatermarkFunction ? (WatermarkFunction) joinFn : null; } + @Override + public TimerFunction getTimerFn() { + return joinFn instanceof TimerFunction ? (TimerFunction) joinFn : null; + } + } diff --git a/samza-core/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java b/samza-core/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java index 06a4f4bfc5..1c8e592f55 100644 --- a/samza-core/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java +++ b/samza-core/src/main/java/org/apache/samza/operators/spec/WindowOperatorSpec.java @@ -20,6 +20,7 @@ package org.apache.samza.operators.spec; import org.apache.samza.operators.functions.FoldLeftFunction; +import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; import org.apache.samza.operators.impl.store.TimeSeriesKeySerde; import org.apache.samza.operators.triggers.AnyTrigger; @@ -122,6 +123,12 @@ public WatermarkFunction getWatermarkFn() { return fn instanceof WatermarkFunction ? (WatermarkFunction) fn : null; } + @Override + public TimerFunction getTimerFn() { + FoldLeftFunction fn = window.getFoldLeftFunction(); + return fn instanceof TimerFunction ? (TimerFunction) fn : null; + } + @Override public Collection getStoreDescriptors() { String storeName = getOpId(); 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 5c5ee84ef1..fb88ddc3d0 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 @@ -43,7 +43,9 @@ import org.apache.samza.processor.StreamProcessor; import org.apache.samza.processor.StreamProcessorLifecycleListener; import org.apache.samza.system.StreamSpec; +import org.apache.samza.task.AsyncStreamTask; import org.apache.samza.task.AsyncStreamTaskFactory; +import org.apache.samza.task.StreamTask; import org.apache.samza.task.StreamTaskFactory; import org.apache.samza.task.TaskFactoryUtil; import org.slf4j.Logger; @@ -144,6 +146,50 @@ public void runTask() { processor.start(); } + public void runSyncTask(StreamTask task) { + JobConfig jobConfig = new JobConfig(this.config); + + if (task == null) { + throw new SamzaException("Task is null"); + } + LOG.info("LocalApplicationRunner will run StreamTask"); + LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener(); + + StreamTaskFactory fac = new StreamTaskFactory() { + @Override + public StreamTask createInstance() { + return task; + } + }; + StreamProcessor processor = new StreamProcessor( + this.config, new HashMap<>(), fac, listener); + numProcessorsToStart.set(1); + listener.setProcessor(processor); + processor.start(); + } + + public void runAsyncTask(AsyncStreamTask task) { + JobConfig jobConfig = new JobConfig(this.config); + + if (task == null) { + throw new SamzaException("Task is null"); + } + LOG.info("LocalApplicationRunner will run StreamTask"); + LocalStreamProcessorLifeCycleListener listener = new LocalStreamProcessorLifeCycleListener(); + + AsyncStreamTaskFactory fac = new AsyncStreamTaskFactory() { + @Override + public AsyncStreamTask createInstance() { + return task; + } + }; + StreamProcessor processor = new StreamProcessor( + this.config, new HashMap<>(), fac, listener); + numProcessorsToStart.set(1); + listener.setProcessor(processor); + processor.start(); + } + @Override public void run(StreamApplication app) { try { diff --git a/samza-core/src/main/java/org/apache/samza/table/TableManager.java b/samza-core/src/main/java/org/apache/samza/table/TableManager.java index c3555f325c..bada304558 100644 --- a/samza-core/src/main/java/org/apache/samza/table/TableManager.java +++ b/samza-core/src/main/java/org/apache/samza/table/TableManager.java @@ -24,13 +24,17 @@ import org.apache.samza.SamzaException; import org.apache.samza.config.Config; import org.apache.samza.config.JavaTableConfig; +import org.apache.samza.container.SamzaContainerContext; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.Serde; -import org.apache.samza.storage.StorageEngine; +import org.apache.samza.task.TaskContext; import org.apache.samza.util.Util; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.base.Preconditions; + + /** * A {@link TableManager} manages tables within a Samza task. For each table, it maintains * the {@link TableSpec} and the {@link TableProvider}. It is used at execution for @@ -61,15 +65,14 @@ static public class TableCtx { // tableId -> TableCtx private final Map tables = new HashMap<>(); - private boolean localTablesInitialized; + private boolean initialized; /** * Construct a table manager instance - * @param config the job configuration + * @param config job configuration * @param serdes Serde instances for tables */ public TableManager(Config config, Map> serdes) { - new JavaTableConfig(config).getTableIds().forEach(tableId -> { // Construct the table provider @@ -91,23 +94,14 @@ public TableManager(Config config, Map> serdes) { } /** - * Initialize all local table - * @param stores stores created locally + * Initialize table providers with container and task contexts + * @param containerContext context for the Samza container + * @param taskContext context for the current task, nullable for global tables */ - public void initLocalTables(Map stores) { - tables.values().forEach(ctx -> { - if (ctx.tableProvider instanceof LocalStoreBackedTableProvider) { - StorageEngine store = stores.get(ctx.tableSpec.getId()); - if (store == null) { - throw new SamzaException(String.format( - "Backing store for table %s was not injected by SamzaContainer", - ctx.tableSpec.getId())); - } - ((LocalStoreBackedTableProvider) ctx.tableProvider).init(store); - } - }); - - localTablesInitialized = true; + public void init(SamzaContainerContext containerContext, TaskContext taskContext) { + Preconditions.checkNotNull(containerContext, "null container context."); + tables.values().forEach(ctx -> ctx.tableProvider.init(containerContext, taskContext)); + initialized = true; } /** @@ -125,18 +119,11 @@ private void addTable(TableSpec tableSpec) { tables.put(tableSpec.getId(), ctx); } - /** - * Start the table manager, internally it starts all tables - */ - public void start() { - tables.values().forEach(ctx -> ctx.tableProvider.start()); - } - /** * Shutdown the table manager, internally it shuts down all tables */ - public void shutdown() { - tables.values().forEach(ctx -> ctx.tableProvider.stop()); + public void close() { + tables.values().forEach(ctx -> ctx.tableProvider.close()); } /** @@ -145,9 +132,10 @@ public void shutdown() { * @return table instance */ public Table getTable(String tableId) { - if (!localTablesInitialized) { - throw new IllegalStateException("Local tables in TableManager not initialized."); + if (!initialized) { + throw new IllegalStateException("TableManager has not been initialized."); } + Preconditions.checkArgument(tables.containsKey(tableId), "Unknown tableId=" + tableId); return tables.get(tableId).tableProvider.getTable(); } } diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/CreditFunction.java b/samza-core/src/main/java/org/apache/samza/table/remote/CreditFunction.java new file mode 100644 index 0000000000..0d30098742 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/table/remote/CreditFunction.java @@ -0,0 +1,36 @@ +/* + * 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.table.remote; + +import java.io.Serializable; +import java.util.function.Function; + +import org.apache.samza.operators.KV; + + +/** + * Function interface for providing rate limiting credits for each table record. + * This interface allows callers to pass in lambda expressions which are otherwise + * non-serializable as-is. + * @param the type of the key + * @param the type of the value + */ +public interface CreditFunction extends Function, Integer>, Serializable { +} \ No newline at end of file diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteReadWriteTable.java b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteReadWriteTable.java new file mode 100644 index 0000000000..a47e3491d8 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteReadWriteTable.java @@ -0,0 +1,178 @@ +/* + * 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.table.remote; + +import java.util.List; + +import org.apache.samza.SamzaException; +import org.apache.samza.container.SamzaContainerContext; +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.Timer; +import org.apache.samza.storage.kv.Entry; +import org.apache.samza.table.ReadWriteTable; +import org.apache.samza.task.TaskContext; +import org.apache.samza.util.RateLimiter; + +import com.google.common.base.Preconditions; + +import static org.apache.samza.table.remote.RemoteTableDescriptor.RL_WRITE_TAG; + + +/** + * Remote store backed read writable table + * + * @param the type of the key in this table + * @param the type of the value in this table + */ +public class RemoteReadWriteTable extends RemoteReadableTable implements ReadWriteTable { + protected final TableWriteFunction writeFn; + protected final CreditFunction writeCreditFn; + protected final boolean rateLimitWrites; + + protected Timer putNs; + protected Timer deleteNs; + protected Timer flushNs; + protected Timer putThrottleNs; // use single timer for all write operations + protected Counter numPuts; + protected Counter numDeletes; + protected Counter numFlushes; + + public RemoteReadWriteTable(String tableId, TableReadFunction readFn, TableWriteFunction writeFn, + RateLimiter ratelimiter, CreditFunction readCreditFn, CreditFunction writeCreditFn) { + super(tableId, readFn, ratelimiter, readCreditFn); + Preconditions.checkNotNull(writeFn, "null write function"); + this.writeFn = writeFn; + this.writeCreditFn = writeCreditFn; + this.rateLimitWrites = rateLimiter != null && rateLimiter.getSupportedTags().contains(RL_WRITE_TAG); + logger.info("Rate limiting is {} for remote write operations", rateLimitWrites ? "enabled" : "disabled"); + } + + /** + * {@inheritDoc} + */ + @Override + public void init(SamzaContainerContext containerContext, TaskContext taskContext) { + super.init(containerContext, taskContext); + putNs = taskContext.getMetricsRegistry().newTimer(groupName, tableId + "-put-ns"); + putThrottleNs = taskContext.getMetricsRegistry().newTimer(groupName, tableId + "-put-throttle-ns"); + deleteNs = taskContext.getMetricsRegistry().newTimer(groupName, tableId + "-delete-ns"); + flushNs = taskContext.getMetricsRegistry().newTimer(groupName, tableId + "-flush-ns"); + numPuts = taskContext.getMetricsRegistry().newCounter(groupName, tableId + "-num-puts"); + numDeletes = taskContext.getMetricsRegistry().newCounter(groupName, tableId + "-num-deletes"); + numFlushes = taskContext.getMetricsRegistry().newCounter(groupName, tableId + "-num-flushes"); + } + + /** + * {@inheritDoc} + */ + @Override + public void put(K key, V value) { + try { + numPuts.inc(); + if (rateLimitWrites) { + throttle(key, value, RL_WRITE_TAG, writeCreditFn, putThrottleNs); + } + long startNs = System.nanoTime(); + writeFn.put(key, value); + putNs.update(System.nanoTime() - startNs); + } catch (Exception e) { + String errMsg = String.format("Failed to put a record, key=%s, value=%s", key, value); + logger.error(errMsg, e); + throw new SamzaException(errMsg, e); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void putAll(List> entries) { + try { + writeFn.putAll(entries); + } catch (Exception e) { + String errMsg = String.format("Failed to put records: %s", entries); + logger.error(errMsg, e); + throw new SamzaException(errMsg, e); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void delete(K key) { + try { + numDeletes.inc(); + if (rateLimitWrites) { + throttle(key, null, RL_WRITE_TAG, writeCreditFn, putThrottleNs); + } + long startNs = System.nanoTime(); + writeFn.delete(key); + deleteNs.update(System.nanoTime() - startNs); + } catch (Exception e) { + String errMsg = String.format("Failed to delete a record, key=%s", key); + logger.error(errMsg, e); + throw new SamzaException(errMsg, e); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void deleteAll(List keys) { + try { + writeFn.deleteAll(keys); + } catch (Exception e) { + String errMsg = String.format("Failed to delete records, keys=%s", keys); + logger.error(errMsg, e); + throw new SamzaException(errMsg, e); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void flush() { + try { + numFlushes.inc(); + if (rateLimitWrites) { + throttle(null, null, RL_WRITE_TAG, writeCreditFn, putThrottleNs); + } + long startNs = System.nanoTime(); + writeFn.flush(); + flushNs.update(System.nanoTime() - startNs); + } catch (Exception e) { + String errMsg = "Failed to flush remote store"; + logger.error(errMsg, e); + throw new SamzaException(errMsg, e); + } + } + + /** + * {@inheritDoc} + */ + @Override + public void close() { + super.close(); + writeFn.close(); + } +} diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteReadableTable.java b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteReadableTable.java new file mode 100644 index 0000000000..ca8e96b909 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteReadableTable.java @@ -0,0 +1,181 @@ +/* + * 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.table.remote; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import org.apache.samza.SamzaException; +import org.apache.samza.container.SamzaContainerContext; +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.Timer; +import org.apache.samza.operators.KV; +import org.apache.samza.table.ReadableTable; +import org.apache.samza.task.TaskContext; +import org.apache.samza.util.RateLimiter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Preconditions; + +import static org.apache.samza.table.remote.RemoteTableDescriptor.RL_READ_TAG; + + +/** + * A Samza {@link org.apache.samza.table.Table} backed by a remote data-store or service. + *

+ * Many stream-processing applications require to look-up data from remote data sources eg: databases, + * web-services, RPC systems to process messages in the stream. Such access to adjunct datasets can be + * naturally modeled as a join between the incoming stream and a {@link RemoteReadableTable}. + *

+ * Example use-cases include: + *

    + *
  • Augmenting a stream of "page-views" with information from a database of user-profiles;
  • + *
  • Scoring page views with impressions services.
  • + *
  • A notifications-system that sends out emails may require a query to an external database to process its message.
  • + *
+ *

+ * A {@link RemoteReadableTable} is meant to be used with a {@link TableReadFunction} and a {@link TableWriteFunction} + * which encapsulate the functionality of reading and writing data to the remote service. These provide a + * pluggable means to specify I/O operations on the table. While the base implementation merely delegates to + * these reader and writer functions, sub-classes of {@link RemoteReadableTable} may provide rich functionality like + * caching or throttling on top of them. + * + * @param the type of the key in this table + * @param the type of the value in this table + */ +public class RemoteReadableTable implements ReadableTable { + protected final String tableId; + protected final Logger logger; + protected final TableReadFunction readFn; + protected final String groupName; + protected final RateLimiter rateLimiter; + protected final CreditFunction readCreditFn; + protected final boolean rateLimitReads; + + protected Timer getNs; + protected Timer getThrottleNs; + protected Counter numGets; + + /** + * Construct a RemoteReadableTable instance + * @param tableId table id + * @param readFn {@link TableReadFunction} for read operations + * @param rateLimiter optional {@link RateLimiter} for throttling reads + * @param readCreditFn function returning a credit to be charged for rate limiting per record + */ + public RemoteReadableTable(String tableId, TableReadFunction readFn, RateLimiter rateLimiter, + CreditFunction readCreditFn) { + Preconditions.checkArgument(tableId != null && !tableId.isEmpty(), "invalid table id"); + Preconditions.checkNotNull(readFn, "null read function"); + this.tableId = tableId; + this.readFn = readFn; + this.rateLimiter = rateLimiter; + this.readCreditFn = readCreditFn; + this.groupName = getClass().getSimpleName(); + this.logger = LoggerFactory.getLogger(groupName + tableId); + this.rateLimitReads = rateLimiter != null && rateLimiter.getSupportedTags().contains(RL_READ_TAG); + logger.info("Rate limiting is {} for remote read operations", rateLimitReads ? "enabled" : "disabled"); + } + + /** + * {@inheritDoc} + */ + @Override + public void init(SamzaContainerContext containerContext, TaskContext taskContext) { + getNs = taskContext.getMetricsRegistry().newTimer(groupName, tableId + "-get-ns"); + getThrottleNs = taskContext.getMetricsRegistry().newTimer(groupName, tableId + "-get-throttle-ns"); + numGets = taskContext.getMetricsRegistry().newCounter(groupName, tableId + "-num-gets"); + } + + /** + * {@inheritDoc} + */ + @Override + public V get(K key) { + try { + numGets.inc(); + if (rateLimitReads) { + throttle(key, null, RL_READ_TAG, readCreditFn, getThrottleNs); + } + long startNs = System.nanoTime(); + V result = readFn.get(key); + getNs.update(System.nanoTime() - startNs); + return result; + } catch (Exception e) { + String errMsg = String.format("Failed to get a record, key=%s", key); + logger.error(errMsg, e); + throw new SamzaException(errMsg, e); + } + } + + /** + * {@inheritDoc} + */ + @Override + public Map getAll(List keys) { + Map result; + try { + result = readFn.getAll(keys); + } catch (Exception e) { + String errMsg = "Failed to get some records"; + logger.error(errMsg, e); + throw new SamzaException(errMsg, e); + } + + if (result == null) { + String errMsg = String.format("Received null records, keys=%s", keys); + logger.error(errMsg); + throw new SamzaException(errMsg); + } + + if (result.size() < keys.size()) { + String errMsg = String.format("Received insufficient number of records (%d), keys=%s", result.size(), keys); + logger.error(errMsg); + throw new SamzaException(errMsg); + } + + return result; + } + + /** + * {@inheritDoc} + */ + @Override + public void close() { + readFn.close(); + } + + /** + * Throttle requests given a table record (key, value) with rate limiter and credit function + * @param key key of the table record (nullable) + * @param value value of the table record (nullable) + * @param tag tag for rate limiter + * @param creditFn mapper function from KV to credits to be charged + * @param timer timer metric to track throttling delays + */ + protected void throttle(K key, V value, String tag, CreditFunction creditFn, Timer timer) { + long startNs = System.nanoTime(); + int credits = (creditFn == null) ? 1 : creditFn.apply(KV.of(key, value)); + rateLimiter.acquire(Collections.singletonMap(tag, credits)); + timer.update(System.nanoTime() - startNs); + } +} diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableDescriptor.java new file mode 100644 index 0000000000..7bc369dbf5 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableDescriptor.java @@ -0,0 +1,214 @@ +/* + * 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.table.remote; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectOutputStream; +import java.util.Base64; +import java.util.HashMap; +import java.util.Map; + +import org.apache.samza.SamzaException; +import org.apache.samza.operators.BaseTableDescriptor; +import org.apache.samza.table.TableSpec; +import org.apache.samza.util.EmbeddedTaggedRateLimiter; +import org.apache.samza.util.RateLimiter; + +import com.google.common.base.Preconditions; + + +/** + * Table descriptor for remote store backed tables + * + * @param the type of the key + * @param the type of the value + */ +public class RemoteTableDescriptor extends BaseTableDescriptor> { + /** + * Tag to be used for provision credits for rate limiting read operations from the remote table. + * Caller must pre-populate the credits with this tag when specifying a custom rate limiter instance + * through {@link RemoteTableDescriptor#withRateLimiter(RateLimiter, CreditFunction, CreditFunction)} + */ + public static final String RL_READ_TAG = "readTag"; + + /** + * Tag to be used for provision credits for rate limiting write operations into the remote table. + * Caller can optionally populate the credits with this tag when specifying a custom rate limiter instance + * through {@link RemoteTableDescriptor#withRateLimiter(RateLimiter, CreditFunction, CreditFunction)} + * and it needs the write functionality. + */ + public static final String RL_WRITE_TAG = "writeTag"; + + // Input support for a specific remote store (required) + private TableReadFunction readFn; + + // Output support for a specific remote store (optional) + private TableWriteFunction writeFn; + + // Rate limiter for client-side throttling; + // can either be constructed indirectly from rates or overridden by withRateLimiter() + private RateLimiter rateLimiter; + + // Rates for constructing the default rate limiter when they are non-zero + private Map tagCreditsMap = new HashMap<>(); + + private CreditFunction readCreditFn; + private CreditFunction writeCreditFn; + + /** + * Construct a table descriptor instance + * @param tableId Id of the table + */ + public RemoteTableDescriptor(String tableId) { + super(tableId); + } + + @Override + public TableSpec getTableSpec() { + validate(); + + Map tableSpecConfig = new HashMap<>(); + generateTableSpecConfig(tableSpecConfig); + + // Serialize and store reader/writer functions + tableSpecConfig.put(RemoteTableProvider.READ_FN, serializeObject("read function", readFn)); + + if (writeFn != null) { + tableSpecConfig.put(RemoteTableProvider.WRITE_FN, serializeObject("write function", writeFn)); + } + + // Serialize the rate limiter if specified + if (!tagCreditsMap.isEmpty()) { + rateLimiter = new EmbeddedTaggedRateLimiter(tagCreditsMap); + } + + if (rateLimiter != null) { + tableSpecConfig.put(RemoteTableProvider.RATE_LIMITER, serializeObject("rate limiter", rateLimiter)); + } + + // Serialize the readCredit and writeCredit functions + if (readCreditFn != null) { + tableSpecConfig.put(RemoteTableProvider.READ_CREDIT_FN, serializeObject( + "read credit function", readCreditFn)); + } + + if (writeCreditFn != null) { + tableSpecConfig.put(RemoteTableProvider.WRITE_CREDIT_FN, serializeObject( + "write credit function", writeCreditFn)); + } + + return new TableSpec(tableId, serde, RemoteTableProviderFactory.class.getName(), tableSpecConfig); + } + + /** + * Use specified TableReadFunction with remote table. + * @param readFn read function instance + * @return this table descriptor instance + */ + public RemoteTableDescriptor withReadFunction(TableReadFunction readFn) { + Preconditions.checkNotNull(readFn, "null read function"); + this.readFn = readFn; + return this; + } + + /** + * Use specified TableWriteFunction with remote table. + * @param writeFn write function instance + * @return this table descriptor instance + */ + public RemoteTableDescriptor withWriteFunction(TableWriteFunction writeFn) { + Preconditions.checkNotNull(writeFn, "null write function"); + this.writeFn = writeFn; + return this; + } + + /** + * Specify a rate limiter along with credit functions to map a table record (as KV) to the amount + * of credits to be charged from the rate limiter for table read and write operations. + * This is an advanced API that provides greater flexibility to throttle each record in the table + * with different number of credits. For most common use-cases eg: limit the number of read/write + * operations, please instead use the {@link RemoteTableDescriptor#withReadRateLimit(int)} and + * {@link RemoteTableDescriptor#withWriteRateLimit(int)}. + * + * @param rateLimiter rate limiter instance to be used for throttling + * @param readCreditFn credit function for rate limiting read operations + * @param writeCreditFn credit function for rate limiting write operations + * @return this table descriptor instance + */ + public RemoteTableDescriptor withRateLimiter(RateLimiter rateLimiter, CreditFunction readCreditFn, + CreditFunction writeCreditFn) { + Preconditions.checkNotNull(rateLimiter, "null read rate limiter"); + this.rateLimiter = rateLimiter; + this.readCreditFn = readCreditFn; + this.writeCreditFn = writeCreditFn; + return this; + } + + /** + * Specify the rate limit for table read operations. If the read rate limit is set with this method + * it is invalid to call {@link RemoteTableDescriptor#withRateLimiter(RateLimiter, CreditFunction, CreditFunction)} + * and vice versa. + * @param creditsPerSec rate limit for read operations; must be positive + * @return this table descriptor instance + */ + public RemoteTableDescriptor withReadRateLimit(int creditsPerSec) { + Preconditions.checkArgument(creditsPerSec > 0, "Max read rate must be a positive number."); + tagCreditsMap.put(RL_READ_TAG, creditsPerSec); + return this; + } + + /** + * Specify the rate limit for table write operations. If the write rate limit is set with this method + * it is invalid to call {@link RemoteTableDescriptor#withRateLimiter(RateLimiter, CreditFunction, CreditFunction)} + * and vice versa. + * @param creditsPerSec rate limit for write operations; must be positive + * @return this table descriptor instance + */ + public RemoteTableDescriptor withWriteRateLimit(int creditsPerSec) { + Preconditions.checkArgument(creditsPerSec > 0, "Max write rate must be a positive number."); + tagCreditsMap.put(RL_WRITE_TAG, creditsPerSec); + return this; + } + + /** + * Helper method to serialize Java objects as Base64 strings + * @param name name of the object (for error reporting) + * @param object object to be serialized + * @return Base64 representation of the object + */ + private String serializeObject(String name, T object) { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos)) { + oos.writeObject(object); + return Base64.getEncoder().encodeToString(baos.toByteArray()); + } catch (IOException e) { + throw new SamzaException("Failed to serialize " + name, e); + } + } + + @Override + protected void validate() { + super.validate(); + Preconditions.checkNotNull(readFn, "TableReadFunction is required."); + Preconditions.checkArgument(rateLimiter == null || tagCreditsMap.isEmpty(), + "Only one of rateLimiter instance or read/write limits can be specified"); + } +} diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableProvider.java b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableProvider.java new file mode 100644 index 0000000000..8b9001ad19 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableProvider.java @@ -0,0 +1,154 @@ +/* + * 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.table.remote; + +import java.io.ByteArrayInputStream; +import java.io.ObjectInputStream; +import java.util.ArrayList; +import java.util.Base64; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.samza.SamzaException; +import org.apache.samza.config.JavaTableConfig; +import org.apache.samza.container.SamzaContainerContext; +import org.apache.samza.table.Table; +import org.apache.samza.table.TableProvider; +import org.apache.samza.table.TableSpec; +import org.apache.samza.task.TaskContext; +import org.apache.samza.util.RateLimiter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Provide for remote table instances + */ +public class RemoteTableProvider implements TableProvider { + private static final Logger LOG = LoggerFactory.getLogger(RemoteTableProvider.class); + + static final String READ_FN = "io.readFn"; + static final String WRITE_FN = "io.writeFn"; + static final String RATE_LIMITER = "io.ratelimiter"; + static final String READ_CREDIT_FN = "io.readCreditFn"; + static final String WRITE_CREDIT_FN = "io.writeCreditFn"; + + private final TableSpec tableSpec; + private final boolean readOnly; + private final List> tables = new ArrayList<>(); + private SamzaContainerContext containerContext; + private TaskContext taskContext; + + public RemoteTableProvider(TableSpec tableSpec) { + this.tableSpec = tableSpec; + readOnly = !tableSpec.getConfig().containsKey(WRITE_FN); + } + + /** + * {@inheritDoc} + */ + @Override + public void init(SamzaContainerContext containerContext, TaskContext taskContext) { + this.containerContext = containerContext; + this.taskContext = taskContext; + } + + /** + * {@inheritDoc} + */ + @Override + public Table getTable() { + RemoteReadableTable table; + TableReadFunction readFn = getReadFn(); + RateLimiter rateLimiter = deserializeObject(RATE_LIMITER); + if (rateLimiter != null) { + rateLimiter.init(containerContext.config, taskContext); + } + CreditFunction readCreditFn = deserializeObject(READ_CREDIT_FN); + if (readOnly) { + table = new RemoteReadableTable(tableSpec.getId(), readFn, rateLimiter, readCreditFn); + } else { + CreditFunction writeCreditFn = deserializeObject(WRITE_CREDIT_FN); + table = new RemoteReadWriteTable(tableSpec.getId(), readFn, getWriteFn(), rateLimiter, readCreditFn, writeCreditFn); + } + table.init(containerContext, taskContext); + tables.add(table); + return table; + } + + /** + * {@inheritDoc} + */ + @Override + public Map generateConfig(Map config) { + Map tableConfig = new HashMap<>(); + + // Insert table_id prefix to config entires + tableSpec.getConfig().forEach((k, v) -> { + String realKey = String.format(JavaTableConfig.TABLE_ID_PREFIX, tableSpec.getId()) + "." + k; + tableConfig.put(realKey, v); + }); + + LOG.info("Generated configuration for table " + tableSpec.getId()); + + return tableConfig; + } + + /** + * {@inheritDoc} + */ + @Override + public void close() { + tables.forEach(t -> t.close()); + } + + private T deserializeObject(String key) { + String entry = tableSpec.getConfig().getOrDefault(key, ""); + if (entry.isEmpty()) { + return null; + } + + try { + byte [] bytes = Base64.getDecoder().decode(entry); + return (T) new ObjectInputStream(new ByteArrayInputStream(bytes)).readObject(); + } catch (Exception e) { + String errMsg = "Failed to deserialize " + key; + throw new SamzaException(errMsg, e); + } + } + + private TableReadFunction getReadFn() { + TableReadFunction readFn = deserializeObject(READ_FN); + if (readFn != null) { + readFn.init(containerContext.config, taskContext); + } + return readFn; + } + + private TableWriteFunction getWriteFn() { + TableWriteFunction writeFn = deserializeObject(WRITE_FN); + if (writeFn != null) { + writeFn.init(containerContext.config, taskContext); + } + return writeFn; + } +} + diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableProviderFactory.java b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableProviderFactory.java new file mode 100644 index 0000000000..0eb88fd1f6 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableProviderFactory.java @@ -0,0 +1,38 @@ +/* + * 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.table.remote; + +import org.apache.samza.table.TableProvider; +import org.apache.samza.table.TableProviderFactory; +import org.apache.samza.table.TableSpec; + +import com.google.common.base.Preconditions; + + +/** + * Factory class for a remote table provider + */ +public class RemoteTableProviderFactory implements TableProviderFactory { + @Override + public TableProvider getTableProvider(TableSpec tableSpec) { + Preconditions.checkNotNull(tableSpec, "null table spec"); + return new RemoteTableProvider(tableSpec); + } +} diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/TableReadFunction.java b/samza-core/src/main/java/org/apache/samza/table/remote/TableReadFunction.java new file mode 100644 index 0000000000..dbd386c50a --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/table/remote/TableReadFunction.java @@ -0,0 +1,66 @@ +/* + * 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.table.remote; + +import java.io.Serializable; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.operators.functions.ClosableFunction; +import org.apache.samza.operators.functions.InitableFunction; + + +/** + * A function object to be used with a {@link RemoteReadableTable} implementation. It encapsulates the functionality + * of reading table record(s) for a provided set of key(s). + * + *

Instances of {@link TableReadFunction} are meant to be serializable. ie. any non-serializable state + * (eg: network sockets) should be marked as transient and recreated inside readObject(). + * + *

Implementations are expected to be thread-safe. + * @param the type of the key in this table + * @param the type of the value in this table + */ +@InterfaceStability.Unstable +public interface TableReadFunction extends Serializable, InitableFunction, ClosableFunction { + /** + * Fetch single table record for a specified {@code key}. This method must be thread-safe. + * @param key key for the table record + * @return table record for the specified {@code key} + */ + V get(K key); + + /** + * Fetch the table {@code records} for specified {@code keys}. This method must be thread-safe. + * @param keys keys for the table records + * @return all records for the specified keys if succeeded; depending on the implementation + * of {@link TableReadFunction#get(Object)} it either returns records for a subset of the + * keys or throws exception when there is any failure. + */ + default Map getAll(Collection keys) { + Map records = new HashMap<>(); + keys.forEach(k -> records.put(k, get(k))); + return records; + } + + // optionally implement readObject() to initialize transient states +} diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/TableWriteFunction.java b/samza-core/src/main/java/org/apache/samza/table/remote/TableWriteFunction.java new file mode 100644 index 0000000000..3fb8fda7fb --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/table/remote/TableWriteFunction.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.table.remote; + +import java.io.Serializable; +import java.util.Collection; +import java.util.List; + +import org.apache.samza.annotation.InterfaceStability; +import org.apache.samza.operators.functions.ClosableFunction; +import org.apache.samza.operators.functions.InitableFunction; +import org.apache.samza.storage.kv.Entry; + + +/** + * A function object to be used with a {@link RemoteReadWriteTable} implementation. It encapsulates the functionality + * of writing table record(s) for a provided set of key(s) to the store. + * + *

Instances of {@link TableWriteFunction} are meant to be serializable. ie. any non-serializable state + * (eg: network sockets) should be marked as transient and recreated inside readObject(). + * + *

Implementations are expected to be thread-safe. + * @param the type of the key in this table + * @param the type of the value in this table + */ +@InterfaceStability.Unstable +public interface TableWriteFunction extends Serializable, InitableFunction, ClosableFunction { + /** + * Store single table {@code record} with specified {@code key}. This method must be thread-safe. + * @param key key for the table record + * @param record table record to be written + */ + void put(K key, V record); + + /** + * Store the table {@code records} with specified {@code keys}. This method must be thread-safe. + * @param records table records to be written + */ + default void putAll(List> records) { + records.forEach(e -> put(e.getKey(), e.getValue())); + } + + /** + * Delete the {@code record} with specified {@code key} from the remote store + * @param key key to the table record to be deleted + */ + void delete(K key); + + /** + * Delete all {@code records} with the specified {@code keys} from the remote store + * @param keys keys for the table records to be written + */ + default void deleteAll(Collection keys) { + keys.stream().forEach(k -> delete(k)); + } + + /** + * Flush the remote store (optional) + */ + default void flush() { + } + + // optionally implement readObject() to initialize transient states +} diff --git a/samza-core/src/main/java/org/apache/samza/task/AsyncRunLoop.java b/samza-core/src/main/java/org/apache/samza/task/AsyncRunLoop.java index b8f48c761c..f4b1d41f39 100644 --- a/samza-core/src/main/java/org/apache/samza/task/AsyncRunLoop.java +++ b/samza-core/src/main/java/org/apache/samza/task/AsyncRunLoop.java @@ -304,6 +304,7 @@ private enum WorkerOp { COMMIT, PROCESS, END_OF_STREAM, + TIMER, NO_OP } @@ -346,6 +347,13 @@ public void run() { } }, commitMs, commitMs, TimeUnit.MILLISECONDS); } + + final SystemTimerScheduler timerFactory = task.context().getTimerScheduler(); + if (timerFactory != null) { + timerFactory.registerListener(() -> { + state.needTimer(); + }); + } } /** @@ -375,6 +383,9 @@ private void run() { case WINDOW: window(); break; + case TIMER: + timer(); + break; case COMMIT: commit(); break; @@ -514,6 +525,39 @@ public void run() { } } + private void timer() { + state.startTimer(); + Runnable timerWorker = new Runnable() { + @Override + public void run() { + try { + ReadableCoordinator coordinator = new ReadableCoordinator(task.taskName()); + + long startTime = clock.nanoTime(); + task.timer(coordinator); + containerMetrics.timerNs().update(clock.nanoTime() - startTime); + + coordinatorRequests.update(coordinator); + state.doneTimer(); + } catch (Throwable t) { + log.error("Task {} timer failed", task.taskName(), t); + abort(t); + } finally { + log.trace("Task {} timer completed", task.taskName()); + resume(); + } + } + }; + + if (threadPool != null) { + log.trace("Task {} timer runs on the thread pool", task.taskName()); + threadPool.submit(timerWorker); + } else { + log.trace("Task {} timer runs on the run loop thread", task.taskName()); + timerWorker.run(); + } + } + /** * Task process completes successfully, update the offsets based on the high-water mark. * Then it will trigger the listener for task state change. @@ -585,10 +629,12 @@ public void onFailure(TaskCallback callback, Throwable t) { private final class AsyncTaskState { private volatile boolean needWindow = false; private volatile boolean needCommit = false; + private volatile boolean needTimer = false; private volatile boolean complete = false; private volatile boolean endOfStream = false; private volatile boolean windowInFlight = false; private volatile boolean commitInFlight = false; + private volatile boolean timerInFlight = false; private final AtomicInteger messagesInFlight = new AtomicInteger(0); private final ArrayDeque pendingEnvelopeQueue; @@ -634,29 +680,28 @@ private boolean isReady() { needCommit = true; } - boolean windowOrCommitInFlight = windowInFlight || commitInFlight; + boolean opInFlight = windowInFlight || commitInFlight || timerInFlight; /* * A task is ready to commit, when task.commit(needCommit) is requested either by user or commit thread * and either of the following conditions are true. - * a) When process, window, commit are not in progress. + * a) When process, window, commit and timer are not in progress. * b) When task.async.commit is true and window, commit are not in progress. */ if (needCommit) { - return (messagesInFlight.get() == 0 || isAsyncCommitEnabled) && !windowOrCommitInFlight; - } else if (needWindow || endOfStream) { + return (messagesInFlight.get() == 0 || isAsyncCommitEnabled) && !opInFlight; + } else if (needWindow || needTimer || endOfStream) { /* - * A task is ready for window operation, when task.window(needWindow) is requested by either user or window thread - * and window, commit are not in progress. + * A task is ready for window, timer or end-of-stream operation. */ - return messagesInFlight.get() == 0 && !windowOrCommitInFlight; + return messagesInFlight.get() == 0 && !opInFlight; } else { /* * A task is ready to process new message, when number of task.process calls in progress < task.max.concurrency * and either of the following conditions are true. - * a) When window, commit are not in progress. - * b) When task.async.commit is true and window is not in progress. + * a) When window, commit and timer are not in progress. + * b) When task.async.commit is true and window and timer are not in progress. */ - return messagesInFlight.get() < maxConcurrency && !windowInFlight && (isAsyncCommitEnabled || !commitInFlight); + return messagesInFlight.get() < maxConcurrency && !windowInFlight && !timerInFlight && (isAsyncCommitEnabled || !commitInFlight); } } @@ -670,6 +715,7 @@ private WorkerOp nextOp() { if (isReady()) { if (needCommit) return WorkerOp.COMMIT; else if (needWindow) return WorkerOp.WINDOW; + else if (needTimer) return WorkerOp.TIMER; else if (endOfStream && pendingEnvelopeQueue.isEmpty()) return WorkerOp.END_OF_STREAM; else if (!pendingEnvelopeQueue.isEmpty()) return WorkerOp.PROCESS; } @@ -684,6 +730,10 @@ private void needCommit() { needCommit = true; } + private void needTimer() { + needTimer = true; + } + private void startWindow() { needWindow = false; windowInFlight = true; @@ -699,6 +749,11 @@ private void startProcess() { taskMetrics.messagesInFlight().set(count); } + private void startTimer() { + needTimer = false; + timerInFlight = true; + } + private void doneCommit() { commitInFlight = false; } @@ -712,6 +767,10 @@ private void doneProcess() { taskMetrics.messagesInFlight().set(count); } + private void doneTimer() { + timerInFlight = false; + } + /** * Insert an PendingEnvelope into the pending envelope queue. * The function will be called in the run loop thread so no synchronization. diff --git a/samza-core/src/main/java/org/apache/samza/task/SystemTimerScheduler.java b/samza-core/src/main/java/org/apache/samza/task/SystemTimerScheduler.java new file mode 100644 index 0000000000..45890581f8 --- /dev/null +++ b/samza-core/src/main/java/org/apache/samza/task/SystemTimerScheduler.java @@ -0,0 +1,154 @@ +/* + * 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.task; + +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +import static com.google.common.base.Preconditions.checkState; + +/** + * Per-task scheduler for keyed timers. + * It does the following things: + * 1) schedules the timer on the {@link ScheduledExecutorService}. + * 2) keeps track of the timers created and timers that are ready. + * 3) triggers listener whenever a timer fires. + */ +public class SystemTimerScheduler { + + /** + * For run loop to listen to timer firing so it can schedule the callbacks. + */ + public interface TimerListener { + void onTimer(); + } + + private final ScheduledExecutorService executor; + private final Map scheduledFutures = new ConcurrentHashMap<>(); + private final Map, TimerCallback> readyTimers = new ConcurrentHashMap<>(); + private TimerListener timerListener; + + public static SystemTimerScheduler create(ScheduledExecutorService executor) { + return new SystemTimerScheduler(executor); + } + + private SystemTimerScheduler(ScheduledExecutorService executor) { + this.executor = executor; + } + + public void setTimer(K key, long timestamp, TimerCallback callback) { + checkState(!scheduledFutures.containsKey(key), + String.format("Duplicate key %s registration for the same timer", key)); + + final long delay = timestamp - System.currentTimeMillis(); + final ScheduledFuture scheduledFuture = executor.schedule(() -> { + readyTimers.put(TimerKey.of(key, timestamp), callback); + + if (timerListener != null) { + timerListener.onTimer(); + } + }, delay > 0 ? delay : 0, TimeUnit.MILLISECONDS); + scheduledFutures.put(key, scheduledFuture); + } + + public void deleteTimer(K key) { + final ScheduledFuture scheduledFuture = scheduledFutures.remove(key); + if (scheduledFuture != null) { + scheduledFuture.cancel(false); + } + } + + void registerListener(TimerListener listener) { + timerListener = listener; + } + + public Map, TimerCallback> removeReadyTimers() { + final Map, TimerCallback> timers = new TreeMap<>(readyTimers); + readyTimers.keySet().removeAll(timers.keySet()); + return timers; + } + + public static class TimerKey implements Comparable> { + private final K key; + private final long time; + + static TimerKey of(K key, long time) { + return new TimerKey<>(key, time); + } + + private TimerKey(K key, long time) { + this.key = key; + this.time = time; + } + + public K getKey() { + return key; + } + + public long getTime() { + return time; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + TimerKey timerKey = (TimerKey) o; + if (time != ((TimerKey) o).time) { + return false; + } + return key.equals(timerKey.key); + } + + @Override + public int hashCode() { + int result = key.hashCode(); + result = 31 * result + Long.valueOf(time).hashCode(); + return result; + } + + @Override + public String toString() { + return "TimerKey{" + + "key=" + key + + ", time='" + time + '\'' + + '}'; + } + + @Override + public int compareTo(TimerKey o) { + final int timeCompare = Long.compare(time, o.time); + if (timeCompare != 0) { + return timeCompare; + } + + return key.hashCode() - o.key.hashCode(); + } + } +} diff --git a/samza-core/src/main/java/org/apache/samza/util/EmbeddedRateLimiter.java b/samza-core/src/main/java/org/apache/samza/util/EmbeddedRateLimiter.java deleted file mode 100644 index 9ccf2f4c4b..0000000000 --- a/samza-core/src/main/java/org/apache/samza/util/EmbeddedRateLimiter.java +++ /dev/null @@ -1,98 +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.util.Map; -import java.util.concurrent.TimeUnit; - -import org.apache.samza.config.Config; -import org.apache.samza.task.TaskContext; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.base.Preconditions; - - -/** - * An embedded rate limiter - */ -public class EmbeddedRateLimiter implements RateLimiter { - - static final private Logger LOGGER = LoggerFactory.getLogger(EmbeddedRateLimiter.class); - - private final int targetRate; - private com.google.common.util.concurrent.RateLimiter rateLimiter; - - public EmbeddedRateLimiter(int creditsPerSecond) { - this.targetRate = creditsPerSecond; - } - - @Override - public void acquire(int numberOfCredits) { - ensureInitialized(); - rateLimiter.acquire(numberOfCredits); - } - - @Override - public int acquire(int numberOfCredits, long timeout, TimeUnit unit) { - ensureInitialized(); - return rateLimiter.tryAcquire(numberOfCredits, timeout, unit) - ? numberOfCredits - : 0; - } - - @Override - public int tryAcquire(int numberOfCredits) { - ensureInitialized(); - return rateLimiter.tryAcquire(numberOfCredits) - ? numberOfCredits - : 0; - } - - @Override - public void acquire(Map tagToCreditsMap) { - throw new IllegalArgumentException("This method is not applicable"); - } - - @Override - public Map acquire(Map tagToCreditsMap, long timeout, TimeUnit unit) { - throw new IllegalArgumentException("This method is not applicable"); - } - - @Override - public Map tryAcquire(Map tagToCreditsMap) { - throw new IllegalArgumentException("This method is not applicable"); - } - - @Override - public void init(Config config, TaskContext taskContext) { - int effectiveRate = targetRate; - if (taskContext != null) { - effectiveRate /= taskContext.getSamzaContainerContext().taskNames.size(); - LOGGER.info(String.format("Effective rate limit for task %s is %d", - taskContext.getTaskName(), effectiveRate)); - } - this.rateLimiter = com.google.common.util.concurrent.RateLimiter.create(effectiveRate); - } - - private void ensureInitialized() { - Preconditions.checkState(rateLimiter != null, "Not initialized"); - } - -} diff --git a/samza-core/src/main/java/org/apache/samza/util/EmbeddedTaggedRateLimiter.java b/samza-core/src/main/java/org/apache/samza/util/EmbeddedTaggedRateLimiter.java index 9c20eeea60..1cf9a9c05a 100644 --- a/samza-core/src/main/java/org/apache/samza/util/EmbeddedTaggedRateLimiter.java +++ b/samza-core/src/main/java/org/apache/samza/util/EmbeddedTaggedRateLimiter.java @@ -20,6 +20,7 @@ import java.util.Collections; import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -36,17 +37,25 @@ /** - * An embedded rate limiter that supports tags + * An embedded rate limiter that supports tags. A default tag will be used if users specifies a simple rate only + * for simple use cases. */ public class EmbeddedTaggedRateLimiter implements RateLimiter { - static final private Logger LOGGER = LoggerFactory.getLogger(EmbeddedTaggedRateLimiter.class); + private static final String DEFAULT_TAG = "default-tag"; + private static final Map DEFAULT_TAG_MAP = Collections.singletonMap(DEFAULT_TAG, 0); private final Map tagToTargetRateMap; private Map tagToRateLimiterMap; + private boolean initialized; + + public EmbeddedTaggedRateLimiter(int creditsPerSecond) { + this(Collections.singletonMap(DEFAULT_TAG, creditsPerSecond)); + } public EmbeddedTaggedRateLimiter(Map tagToCreditsPerSecondMap) { Preconditions.checkArgument(tagToCreditsPerSecondMap.size() > 0, "Map of tags can't be empty"); + tagToCreditsPerSecondMap.values().forEach(c -> Preconditions.checkArgument(c >= 0, "Credits must be non-negative")); this.tagToTargetRateMap = tagToCreditsPerSecondMap; } @@ -72,39 +81,28 @@ public Map acquire(Map tagToCreditsMap, long t int availableCredits = rateLimiter.tryAcquire(requiredCredits, remainingTimeoutInNanos, NANOSECONDS) ? requiredCredits : 0; - return new ImmutablePair(tag, availableCredits); + return new ImmutablePair<>(tag, availableCredits); }) .collect(Collectors.toMap(ImmutablePair::getKey, ImmutablePair::getValue)); } @Override - public Map tryAcquire(Map tagToCreditsMap) { - ensureTagsAreValid(tagToCreditsMap); - return tagToCreditsMap.entrySet().stream() - .map(e -> { - String tag = e.getKey(); - int requiredCredits = e.getValue(); - int availableCredits = tagToRateLimiterMap.get(tag).tryAcquire(requiredCredits) - ? requiredCredits - : 0; - return new ImmutablePair(tag, availableCredits); - }) - .collect(Collectors.toMap(ImmutablePair::getKey, ImmutablePair::getValue)); + public Set getSupportedTags() { + return Collections.unmodifiableSet(tagToRateLimiterMap.keySet()); } @Override public void acquire(int numberOfCredits) { - throw new IllegalArgumentException("This method is not applicable"); + ensureTagsAreValid(DEFAULT_TAG_MAP); + tagToRateLimiterMap.get(DEFAULT_TAG).acquire(numberOfCredits); } @Override public int acquire(int numberOfCredit, long timeout, TimeUnit unit) { - throw new IllegalArgumentException("This method is not applicable"); - } - - @Override - public int tryAcquire(int numberOfCredit) { - throw new IllegalArgumentException("This method is not applicable"); + ensureTagsAreValid(DEFAULT_TAG_MAP); + return tagToRateLimiterMap.get(DEFAULT_TAG).tryAcquire(numberOfCredit, timeout, unit) + ? numberOfCredit + : 0; } @Override @@ -118,15 +116,15 @@ public void init(Config config, TaskContext taskContext) { LOGGER.info(String.format("Effective rate limit for task %s and tag %s is %d", taskContext.getTaskName(), tag, effectiveRate)); } - return new ImmutablePair( - tag, com.google.common.util.concurrent.RateLimiter.create(effectiveRate)); + return new ImmutablePair<>(tag, com.google.common.util.concurrent.RateLimiter.create(effectiveRate)); }) .collect(Collectors.toMap(ImmutablePair::getKey, ImmutablePair::getValue)) ); + initialized = true; } private void ensureInitialized() { - Preconditions.checkState(tagToRateLimiterMap != null, "Not initialized"); + Preconditions.checkState(initialized, "Not initialized"); } private void ensureTagsAreValid(Map tagMap) { diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinator.java b/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinator.java index 00eeeaefee..407291a806 100644 --- a/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinator.java +++ b/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinator.java @@ -195,11 +195,12 @@ void doOnProcessorChange(List processors) { Set uniqueProcessorIds = new HashSet(currentProcessorIds); if (currentProcessorIds.size() != uniqueProcessorIds.size()) { - LOG.info("Processors: {} has duplicates. Not generating job model.", currentProcessorIds); + LOG.info("Processors: {} has duplicates. Not generating JobModel.", currentProcessorIds); return; } // Generate the JobModel + LOG.info("Generating new JobModel with processors: {}.", currentProcessorIds); JobModel jobModel = generateNewJobModel(currentProcessorIds); // Create checkpoint and changelog streams if they don't exist @@ -217,7 +218,7 @@ void doOnProcessorChange(List processors) { // Assign the next version of JobModel String currentJMVersion = zkUtils.getJobModelVersion(); String nextJMVersion = zkUtils.getNextJobModelVersion(currentJMVersion); - LOG.info("pid=" + processorId + "Generated new Job Model. Version = " + nextJMVersion); + LOG.info("pid=" + processorId + "Generated new JobModel with version: " + nextJMVersion + " and processors: " + currentProcessorIds); // Publish the new job model zkUtils.publishJobModel(nextJMVersion, jobModel); diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkUtils.java b/samza-core/src/main/java/org/apache/samza/zk/ZkUtils.java index 300fff661a..43f7d9ccd4 100644 --- a/samza-core/src/main/java/org/apache/samza/zk/ZkUtils.java +++ b/samza-core/src/main/java/org/apache/samza/zk/ZkUtils.java @@ -189,15 +189,18 @@ private boolean isValidRegisteredProcessor(final ProcessorNode processor) { * Fetches all the ephemeral processor nodes of a standalone job from zookeeper. * @return a list of {@link ProcessorNode}, where each ProcessorNode represents a registered stream processor. */ - private List getAllProcessorNodes() { + List getAllProcessorNodes() { List processorZNodes = getSortedActiveProcessorsZnodes(); LOG.debug("Active ProcessorZNodes in zookeeper: {}.", processorZNodes); - return processorZNodes.stream() - .map(processorZNode -> { - String ephemeralProcessorPath = String.format("%s/%s", keyBuilder.getProcessorsPath(), processorZNode); - String data = readProcessorData(ephemeralProcessorPath); - return new ProcessorNode(new ProcessorData(data), ephemeralProcessorPath); - }).collect(Collectors.toList()); + List processorNodes = new ArrayList<>(); + for (String processorZNode: processorZNodes) { + String ephemeralProcessorPath = String.format("%s/%s", keyBuilder.getProcessorsPath(), processorZNode); + String data = readProcessorData(ephemeralProcessorPath); + if (data != null) { + processorNodes.add(new ProcessorNode(new ProcessorData(data), ephemeralProcessorPath)); + } + } + return processorNodes; } /** @@ -221,12 +224,10 @@ public List getSortedActiveProcessorsZnodes() { * @throws SamzaException when fullPath doesn't exist in zookeeper * or problems with connecting to zookeeper. */ - String readProcessorData(String fullPath) { + private String readProcessorData(String fullPath) { try { - String data = zkClient.readData(fullPath, false); - if (metrics != null) { - metrics.reads.inc(); - } + String data = zkClient.readData(fullPath, true); + metrics.reads.inc(); return data; } catch (Exception e) { throw new SamzaException(String.format("Cannot read ZK node: %s", fullPath), e); @@ -252,7 +253,10 @@ public List getActiveProcessorsIDs(List znodeIds) { if (znodeIds.size() > 0) { for (String child : znodeIds) { String fullPath = String.format("%s/%s", processorPath, child); - processorIds.add(new ProcessorData(readProcessorData(fullPath)).getProcessorId()); + String processorData = readProcessorData(fullPath); + if (processorData != null) { + processorIds.add(new ProcessorData(processorData).getProcessorId()); + } } Collections.sort(processorIds); LOG.info("Found these children - " + znodeIds); @@ -589,7 +593,7 @@ void deleteOldVersionPath(String path, List zNodeIds, int numVersionsToL /** * Represents zookeeper processor node. */ - private static class ProcessorNode { + static class ProcessorNode { private final ProcessorData processorData; // Ex: /test/processors/0000000000 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 bef5b41b82..7cc1924572 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 @@ -24,7 +24,7 @@ import java.net.{URL, UnknownHostException} import java.nio.file.Path import java.util import java.util.Base64 -import java.util.concurrent.{ExecutorService, Executors, TimeUnit} +import java.util.concurrent.{ScheduledExecutorService, ExecutorService, Executors, TimeUnit} import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.samza.checkpoint.{CheckpointListener, CheckpointManagerFactory, OffsetManager, OffsetManagerMetrics} @@ -436,6 +436,8 @@ object SamzaContainer extends Logging { val storeWatchPaths = new util.HashSet[Path]() + val timerExecutor = Executors.newSingleThreadScheduledExecutor + val taskInstances: Map[TaskName, TaskInstance] = containerModel.getTasks.values.asScala.map(taskModel => { debug("Setting up task instance: %s" format taskModel) @@ -536,9 +538,8 @@ object SamzaContainer extends Logging { new SystemClock) val tableManager = new TableManager(config, serdes.asJava) - tableManager.initLocalTables(taskStores.asJava) - info("Got table manager"); + info("Got table manager") val systemStreamPartitions = taskModel .getSystemStreamPartitions @@ -563,7 +564,8 @@ object SamzaContainer extends Logging { systemStreamPartitions = systemStreamPartitions, exceptionHandler = TaskInstanceExceptionHandler(taskInstanceMetrics, config), jobModel = jobModel, - streamMetadataCache = streamMetadataCache) + streamMetadataCache = streamMetadataCache, + timerExecutor = timerExecutor) val taskInstance = createTaskInstance(task) @@ -636,7 +638,8 @@ object SamzaContainer extends Logging { jvm = jvm, diskSpaceMonitor = diskSpaceMonitor, hostStatisticsMonitor = memoryStatisticsMonitor, - taskThreadPool = taskThreadPool) + taskThreadPool = taskThreadPool, + timerExecutor = timerExecutor) } } @@ -656,7 +659,8 @@ class SamzaContainer( securityManager: SecurityManager = null, reporters: Map[String, MetricsReporter] = Map(), jvm: JvmMetrics = null, - taskThreadPool: ExecutorService = null) extends Runnable with Logging { + taskThreadPool: ExecutorService = null, + timerExecutor: ScheduledExecutorService = Executors.newSingleThreadScheduledExecutor) extends Runnable with Logging { val shutdownMs = containerContext.config.getShutdownMs.getOrElse(TaskConfigJava.DEFAULT_TASK_SHUTDOWN_MS) var shutdownHookThread: Thread = null @@ -1003,6 +1007,18 @@ class SamzaContainer( } } + if (timerExecutor != null) { + info("Shutting down timer executor") + try { + timerExecutor.shutdown() + if (timerExecutor.awaitTermination(shutdownMs, TimeUnit.MILLISECONDS)) { + timerExecutor.shutdownNow() + } + } catch { + case e: Exception => error("Ignoring exception shutting down timer executor", e) + } + } + if (isAutoCommitEnabled) { info("Committing offsets for all task instances") taskInstances.values.foreach(_.commit) diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala index d080939de5..c1229565ad 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainerMetrics.scala @@ -29,11 +29,13 @@ class SamzaContainerMetrics( val commits = newCounter("commit-calls") val windows = newCounter("window-calls") + val timers = newCounter("timer-calls") val processes = newCounter("process-calls") val envelopes = newCounter("process-envelopes") val nullEnvelopes = newCounter("process-null-envelopes") val chooseNs = newTimer("choose-ns") val windowNs = newTimer("window-ns") + val timerNs = newTimer("timer-ns") val processNs = newTimer("process-ns") val commitNs = newTimer("commit-ns") val blockNs = newTimer("block-ns") diff --git a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala index c7d76c2096..3ac37c65ed 100644 --- a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala +++ b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala @@ -20,12 +20,15 @@ package org.apache.samza.container +import java.util.concurrent.ScheduledExecutorService + import org.apache.samza.SamzaException import org.apache.samza.checkpoint.OffsetManager import org.apache.samza.config.Config import org.apache.samza.config.StreamConfig.Config2Stream import org.apache.samza.job.model.JobModel import org.apache.samza.metrics.MetricsReporter +import org.apache.samza.operators.functions.TimerFunction import org.apache.samza.storage.TaskStorageManager import org.apache.samza.system._ import org.apache.samza.table.TableManager @@ -33,6 +36,7 @@ import org.apache.samza.task._ import org.apache.samza.util.Logging import scala.collection.JavaConverters._ +import scala.collection.JavaConversions._ class TaskInstance( val task: Any, @@ -50,7 +54,8 @@ class TaskInstance( val systemStreamPartitions: Set[SystemStreamPartition] = Set(), val exceptionHandler: TaskInstanceExceptionHandler = new TaskInstanceExceptionHandler, jobModel: JobModel = null, - streamMetadataCache: StreamMetadataCache = null) extends Logging { + streamMetadataCache: StreamMetadataCache = null, + timerExecutor : ScheduledExecutorService = null) extends Logging { val isInitableTask = task.isInstanceOf[InitableTask] val isWindowableTask = task.isInstanceOf[WindowableTask] val isEndOfStreamListenerTask = task.isInstanceOf[EndOfStreamListenerTask] @@ -58,7 +63,7 @@ class TaskInstance( val isAsyncTask = task.isInstanceOf[AsyncStreamTask] val context = new TaskContextImpl(taskName, metrics, containerContext, systemStreamPartitions.asJava, offsetManager, - storageManager, tableManager, jobModel, streamMetadataCache) + storageManager, tableManager, jobModel, streamMetadataCache, timerExecutor) // store the (ssp -> if this ssp is catched up) mapping. "catched up" // means the same ssp in other taskInstances have the same offset as @@ -95,7 +100,7 @@ class TaskInstance( if (tableManager != null) { debug("Starting table manager for taskName: %s" format taskName) - tableManager.start + tableManager.init(containerContext, context) } else { debug("Skipping table manager initialization for taskName: %s" format taskName) } @@ -185,6 +190,16 @@ class TaskInstance( } } + def timer(coordinator: ReadableCoordinator) { + trace("Timer for taskName: %s" format taskName) + + exceptionHandler.maybeHandle { + context.getTimerScheduler.removeReadyTimers().entrySet().foreach { entry => + entry.getValue.asInstanceOf[TimerCallback[Any]].onTimer(entry.getKey.getKey, collector, coordinator) + } + } + } + def commit { metrics.commits.inc @@ -229,7 +244,7 @@ class TaskInstance( if (tableManager != null) { debug("Shutting down table manager for taskName: %s" format taskName) - tableManager.shutdown + tableManager.close } else { debug("Skipping table manager shutdown for taskName: %s" format taskName) } diff --git a/samza-core/src/main/scala/org/apache/samza/system/chooser/BootstrappingChooser.scala b/samza-core/src/main/scala/org/apache/samza/system/chooser/BootstrappingChooser.scala index 212ec05bad..ffc0c90c07 100644 --- a/samza-core/src/main/scala/org/apache/samza/system/chooser/BootstrappingChooser.scala +++ b/samza-core/src/main/scala/org/apache/samza/system/chooser/BootstrappingChooser.scala @@ -266,9 +266,17 @@ class BootstrappingChooser( trace("Check %s offset %s against %s for %s." format (offsetType, offset, offsetToCheck, systemStreamPartition)) - // The SSP is no longer lagging if the envelope's offset equals the + // Let's compare offset of the chosen message with offsetToCheck. + val comparatorResult: Integer = if (offset == null || offsetToCheck == null) { + -1 + } else { + val systemAdmin = systemAdmins.getSystemAdmin(systemStreamPartition.getSystem) + systemAdmin.offsetComparator(offset, offsetToCheck) + } + + // The SSP is no longer lagging if the envelope's offset is greater than or equal to the // latest offset. - if (offset != null && offset.equals(offsetToCheck)) { + if (comparatorResult != null && comparatorResult.intValue() >= 0) { laggingSystemStreamPartitions -= systemStreamPartition systemStreamLagCounts += systemStream -> (systemStreamLagCounts(systemStream) - 1) diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java index a4f0354bb7..249ff09f74 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImpl.java @@ -28,6 +28,7 @@ import org.apache.samza.metrics.MetricsRegistryMap; import org.apache.samza.metrics.ReadableMetricsRegistry; import org.apache.samza.metrics.Timer; +import org.apache.samza.operators.functions.TimerFunction; import org.apache.samza.operators.functions.WatermarkFunction; import org.apache.samza.operators.spec.OperatorSpec; import org.apache.samza.task.MessageCollector; @@ -217,6 +218,11 @@ private static class TestOpSpec extends OperatorSpec { public WatermarkFunction getWatermarkFn() { return null; } + + @Override + public TimerFunction getTimerFn() { + return null; + } } public static Set getNextOperators(OperatorImpl op) { diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java index 1448f7986b..2d8d1eb57c 100644 --- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java +++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java @@ -220,7 +220,7 @@ public void testMergeChain() { new OperatorImplGraph(streamGraph, mock(Config.class), mockTaskContext, mock(Clock.class)); // verify that the DAG after merge is only traversed & initialized once - verify(mockMapFunction, times(1)).init(any(Config.class), any(TaskContextImpl.class)); + verify(mockMapFunction, times(1)).init(any(Config.class), any(TaskContext.class)); } @Test @@ -249,7 +249,7 @@ public void testJoinChain() { new OperatorImplGraph(streamGraph, mockConfig, mockTaskContext, mock(Clock.class)); // verify that join function is initialized once. - verify(mockJoinFunction, times(1)).init(any(Config.class), any(TaskContextImpl.class)); + verify(mockJoinFunction, times(1)).init(any(Config.class), any(TaskContext.class)); InputOperatorImpl inputOpImpl1 = opImplGraph.getInputOperator(new SystemStream("input-system", "input-stream1")); InputOperatorImpl inputOpImpl2 = opImplGraph.getInputOperator(new SystemStream("input-system", "input-stream2")); diff --git a/samza-core/src/test/java/org/apache/samza/table/TestTableManager.java b/samza-core/src/test/java/org/apache/samza/table/TestTableManager.java index df5b9e5746..24178d0479 100644 --- a/samza-core/src/test/java/org/apache/samza/table/TestTableManager.java +++ b/samza-core/src/test/java/org/apache/samza/table/TestTableManager.java @@ -27,11 +27,13 @@ import org.apache.samza.config.JavaTableConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.config.SerializerConfig; +import org.apache.samza.container.SamzaContainerContext; import org.apache.samza.serializers.IntegerSerde; import org.apache.samza.serializers.Serde; import org.apache.samza.serializers.SerializableSerde; import org.apache.samza.serializers.StringSerde; import org.apache.samza.storage.StorageEngine; +import org.apache.samza.task.TaskContext; import org.junit.Test; import junit.framework.Assert; @@ -49,13 +51,13 @@ public class TestTableManager { public static class DummyTableProviderFactory implements TableProviderFactory { - static Table table; - static LocalStoreBackedTableProvider tableProvider; + static ReadableTable table; + static TableProvider tableProvider; @Override public TableProvider getTableProvider(TableSpec tableSpec) { - table = mock(Table.class); - tableProvider = mock(LocalStoreBackedTableProvider.class); + table = mock(ReadableTable.class); + tableProvider = mock(TableProvider.class); when(tableProvider.getTable()).thenReturn(table); return tableProvider; } @@ -120,10 +122,10 @@ private void doTestInit(Map map) { }); TableManager tableManager = new TableManager(new MapConfig(map), serdeMap); - tableManager.initLocalTables(storageEngines); + tableManager.init(mock(SamzaContainerContext.class), mock(TaskContext.class)); Table table = tableManager.getTable(TABLE_ID); - verify(DummyTableProviderFactory.tableProvider, times(1)).init(anyObject()); + verify(DummyTableProviderFactory.tableProvider, times(1)).init(anyObject(), anyObject()); Assert.assertEquals(DummyTableProviderFactory.table, table); Map ctxMap = getFieldValue(tableManager, "tables"); diff --git a/samza-core/src/test/java/org/apache/samza/table/remote/TestRemoteTableDescriptor.java b/samza-core/src/test/java/org/apache/samza/table/remote/TestRemoteTableDescriptor.java new file mode 100644 index 0000000000..acf3d619e5 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/table/remote/TestRemoteTableDescriptor.java @@ -0,0 +1,244 @@ +/* + * 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.table.remote; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import org.apache.samza.container.SamzaContainerContext; +import org.apache.samza.container.TaskName; +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.metrics.Timer; +import org.apache.samza.operators.KV; +import org.apache.samza.table.Table; +import org.apache.samza.table.TableSpec; +import org.apache.samza.task.TaskContext; +import org.apache.samza.util.EmbeddedTaggedRateLimiter; +import org.apache.samza.util.RateLimiter; +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.samza.table.remote.RemoteTableDescriptor.RL_READ_TAG; +import static org.apache.samza.table.remote.RemoteTableDescriptor.RL_WRITE_TAG; +import static org.mockito.Matchers.anyMap; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + + +public class TestRemoteTableDescriptor { + private void doTestSerialize(RateLimiter rateLimiter, + CreditFunction readCredFn, + CreditFunction writeCredFn) { + RemoteTableDescriptor desc = new RemoteTableDescriptor("1"); + desc.withReadFunction(mock(TableReadFunction.class)); + desc.withWriteFunction(mock(TableWriteFunction.class)); + if (rateLimiter != null) { + desc.withRateLimiter(rateLimiter, readCredFn, writeCredFn); + } else { + desc.withReadRateLimit(100); + desc.withWriteRateLimit(200); + } + TableSpec spec = desc.getTableSpec(); + Assert.assertTrue(spec.getConfig().containsKey(RemoteTableProvider.RATE_LIMITER)); + Assert.assertEquals(readCredFn != null, spec.getConfig().containsKey(RemoteTableProvider.READ_CREDIT_FN)); + Assert.assertEquals(writeCredFn != null, spec.getConfig().containsKey(RemoteTableProvider.WRITE_CREDIT_FN)); + } + + @Test + public void testSerializeSimple() { + doTestSerialize(null, null, null); + } + + @Test + public void testSerializeWithLimiter() { + doTestSerialize(mock(RateLimiter.class), null, null); + } + + @Test + public void testSerializeWithLimiterAndReadCredFn() { + doTestSerialize(mock(RateLimiter.class), kv -> 1, null); + } + + @Test + public void testSerializeWithLimiterAndWriteCredFn() { + doTestSerialize(mock(RateLimiter.class), null, kv -> 1); + } + + @Test + public void testSerializeWithLimiterAndReadWriteCredFns() { + doTestSerialize(mock(RateLimiter.class), kv -> 1, kv -> 1); + } + + @Test + public void testSerializeNullWriteFunction() { + RemoteTableDescriptor desc = new RemoteTableDescriptor("1"); + desc.withReadFunction(mock(TableReadFunction.class)); + TableSpec spec = desc.getTableSpec(); + Assert.assertTrue(spec.getConfig().containsKey(RemoteTableProvider.READ_FN)); + Assert.assertFalse(spec.getConfig().containsKey(RemoteTableProvider.WRITE_FN)); + } + + @Test(expected = NullPointerException.class) + public void testSerializeNullReadFunction() { + RemoteTableDescriptor desc = new RemoteTableDescriptor("1"); + TableSpec spec = desc.getTableSpec(); + Assert.assertTrue(spec.getConfig().containsKey(RemoteTableProvider.READ_FN)); + } + + @Test(expected = IllegalArgumentException.class) + public void testSpecifyBothRateAndRateLimiter() { + RemoteTableDescriptor desc = new RemoteTableDescriptor("1"); + desc.withReadFunction(mock(TableReadFunction.class)); + desc.withReadRateLimit(100); + desc.withRateLimiter(mock(RateLimiter.class), null, null); + desc.getTableSpec(); + } + + private TaskContext createMockTaskContext() { + MetricsRegistry metricsRegistry = mock(MetricsRegistry.class); + doReturn(mock(Timer.class)).when(metricsRegistry).newTimer(anyString(), anyString()); + doReturn(mock(Counter.class)).when(metricsRegistry).newCounter(anyString(), anyString()); + TaskContext taskContext = mock(TaskContext.class); + doReturn(metricsRegistry).when(taskContext).getMetricsRegistry(); + SamzaContainerContext containerCtx = new SamzaContainerContext( + "1", null, Collections.singleton(new TaskName("MyTask")), null); + doReturn(containerCtx).when(taskContext).getSamzaContainerContext(); + return taskContext; + } + + static class CountingCreditFunction implements CreditFunction { + int numCalls = 0; + @Override + public Integer apply(KV kv) { + numCalls++; + return 1; + } + } + + private void doTestDeserializeReadFunctionAndLimiter(boolean rateOnly, boolean rlGets, boolean rlPuts) { + int numRateLimitOps = (rlGets ? 1 : 0) + (rlPuts ? 1 : 0); + RemoteTableDescriptor desc = new RemoteTableDescriptor("1"); + desc.withReadFunction(mock(TableReadFunction.class)); + desc.withWriteFunction(mock(TableWriteFunction.class)); + if (rateOnly) { + if (rlGets) { + desc.withReadRateLimit(1000); + } + if (rlPuts) { + desc.withWriteRateLimit(2000); + } + } else { + if (numRateLimitOps > 0) { + Map tagCredits = new HashMap<>(); + if (rlGets) { + tagCredits.put(RL_READ_TAG, 1000); + } + if (rlPuts) { + tagCredits.put(RL_WRITE_TAG, 2000); + } + + // Spy the rate limiter to verify call count + RateLimiter rateLimiter = spy(new EmbeddedTaggedRateLimiter(tagCredits)); + desc.withRateLimiter(rateLimiter, new CountingCreditFunction(), new CountingCreditFunction()); + } + } + + TableSpec spec = desc.getTableSpec(); + RemoteTableProvider provider = new RemoteTableProvider(spec); + provider.init(mock(SamzaContainerContext.class), createMockTaskContext()); + Table table = provider.getTable(); + Assert.assertTrue(table instanceof RemoteReadWriteTable); + RemoteReadWriteTable rwTable = (RemoteReadWriteTable) table; + Assert.assertNotNull(rwTable.readFn); + Assert.assertNotNull(rwTable.writeFn); + if (numRateLimitOps > 0) { + Assert.assertNotNull(rwTable.rateLimiter); + } + + // Verify rate limiter usage + if (numRateLimitOps > 0) { + rwTable.get("xxx"); + rwTable.put("yyy", "zzz"); + + if (!rateOnly) { + verify(rwTable.rateLimiter, times(numRateLimitOps)).acquire(anyMap()); + + CountingCreditFunction readCreditFn = (CountingCreditFunction) rwTable.readCreditFn; + CountingCreditFunction writeCreditFn = (CountingCreditFunction) rwTable.writeCreditFn; + + Assert.assertNotNull(readCreditFn); + Assert.assertNotNull(writeCreditFn); + + Assert.assertEquals(readCreditFn.numCalls, rlGets ? 1 : 0); + Assert.assertEquals(writeCreditFn.numCalls, rlPuts ? 1 : 0); + } else { + Assert.assertTrue(rwTable.rateLimiter instanceof EmbeddedTaggedRateLimiter); + Assert.assertEquals(rwTable.rateLimiter.getSupportedTags().size(), numRateLimitOps); + if (rlGets) { + Assert.assertTrue(rwTable.rateLimiter.getSupportedTags().contains(RL_READ_TAG)); + } + if (rlPuts) { + Assert.assertTrue(rwTable.rateLimiter.getSupportedTags().contains(RL_WRITE_TAG)); + } + } + } + } + + @Test + public void testDeserializeReadFunctionNoRateLimit() { + doTestDeserializeReadFunctionAndLimiter(false, false, false); + } + + @Test + public void testDeserializeReadFunctionAndLimiterWrite() { + doTestDeserializeReadFunctionAndLimiter(false, false, true); + } + + @Test + public void testDeserializeReadFunctionAndLimiterRead() { + doTestDeserializeReadFunctionAndLimiter(false, true, false); + } + + @Test + public void testDeserializeReadFunctionAndLimiterReadWrite() { + doTestDeserializeReadFunctionAndLimiter(false, true, true); + } + + @Test + public void testDeserializeReadFunctionAndLimiterRateOnlyWrite() { + doTestDeserializeReadFunctionAndLimiter(true, false, true); + } + + @Test + public void testDeserializeReadFunctionAndLimiterRateOnlyRead() { + doTestDeserializeReadFunctionAndLimiter(true, true, false); + } + + @Test + public void testDeserializeReadFunctionAndLimiterRateOnlyReadWrite() { + doTestDeserializeReadFunctionAndLimiter(true, true, true); + } +} diff --git a/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java b/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java index 28a4f8b513..7f5461486d 100644 --- a/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java +++ b/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java @@ -86,7 +86,8 @@ TaskInstance createTaskInstance(AsyncStreamTask task, TaskName taskName, SystemS scala.collection.immutable.Set sspSet = JavaConverters.asScalaSetConverter(Collections.singleton(ssp)).asScala().toSet(); return new TaskInstance(task, taskName, mock(Config.class), taskInstanceMetrics, null, consumers, mock(TaskInstanceCollector.class), mock(SamzaContainerContext.class), - manager, null, null, null, sspSet, new TaskInstanceExceptionHandler(taskInstanceMetrics, new scala.collection.immutable.HashSet()), null, null); + manager, null, null, null, sspSet, new TaskInstanceExceptionHandler(taskInstanceMetrics, + new scala.collection.immutable.HashSet()), null, null, null); } TaskInstance createTaskInstance(AsyncStreamTask task, TaskName taskName, SystemStreamPartition ssp) { diff --git a/samza-core/src/test/java/org/apache/samza/task/TestSystemTimerScheduler.java b/samza-core/src/test/java/org/apache/samza/task/TestSystemTimerScheduler.java new file mode 100644 index 0000000000..dd08121728 --- /dev/null +++ b/samza-core/src/test/java/org/apache/samza/task/TestSystemTimerScheduler.java @@ -0,0 +1,176 @@ +/* + * 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.task; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.anyBoolean; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class TestSystemTimerScheduler { + + private ScheduledExecutorService createExecutorService() { + ScheduledExecutorService service = mock(ScheduledExecutorService.class); + when(service.schedule((Runnable) anyObject(), anyLong(), anyObject())).thenAnswer(invocation -> { + Object[] args = invocation.getArguments(); + Runnable runnable = (Runnable) args[0]; + runnable.run(); + return mock(ScheduledFuture.class); + }); + return service; + } + + private void fireTimers(SystemTimerScheduler factory) { + factory.removeReadyTimers().entrySet().forEach(entry -> { + entry.getValue().onTimer(entry.getKey().getKey(), mock(MessageCollector.class), mock(TaskCoordinator.class)); + }); + } + + @Test + public void testSingleTimer() { + SystemTimerScheduler scheduler = SystemTimerScheduler.create(createExecutorService()); + List results = new ArrayList<>(); + scheduler.setTimer("single-timer", 1, (key, collector, coordinator) -> { + results.add(key); + }); + + fireTimers(scheduler); + + assertTrue(results.size() == 1); + assertEquals(results.get(0), "single-timer"); + } + + @Test + public void testMultipleTimers() { + SystemTimerScheduler scheduler = SystemTimerScheduler.create(createExecutorService()); + List results = new ArrayList<>(); + scheduler.setTimer("multiple-timer-3", 3, (key, collector, coordinator) -> { + results.add(key + ":3"); + }); + scheduler.setTimer("multiple-timer-2", 2, (key, collector, coordinator) -> { + results.add(key + ":2"); + }); + scheduler.setTimer("multiple-timer-1", 1, (key, collector, coordinator) -> { + results.add(key + ":1"); + }); + + fireTimers(scheduler); + + assertTrue(results.size() == 3); + assertEquals(results.get(0), "multiple-timer-1:1"); + assertEquals(results.get(1), "multiple-timer-2:2"); + assertEquals(results.get(2), "multiple-timer-3:3"); + } + + @Test + public void testMultipleKeys() { + Object key1 = new Object(); + Object key2 = new Object(); + List results = new ArrayList<>(); + + SystemTimerScheduler scheduler = SystemTimerScheduler.create(createExecutorService()); + scheduler.setTimer(key1, 2, (key, collector, coordinator) -> { + assertEquals(key, key1); + results.add("key1:2"); + }); + scheduler.setTimer(key2, 1, (key, collector, coordinator) -> { + assertEquals(key, key2); + results.add("key2:1"); + }); + + fireTimers(scheduler); + + assertTrue(results.size() == 2); + assertEquals(results.get(0), "key2:1"); + assertEquals(results.get(1), "key1:2"); + } + + @Test + public void testMultipleKeyTypes() { + String key1 = "key"; + Long key2 = Long.MAX_VALUE; + List results = new ArrayList<>(); + + SystemTimerScheduler scheduler = SystemTimerScheduler.create(createExecutorService()); + scheduler.setTimer(key1, 1, (key, collector, coordinator) -> { + assertEquals(key, key1); + results.add("key:1"); + }); + scheduler.setTimer(key2, 2, (key, collector, coordinator) -> { + assertEquals(key.longValue(), Long.MAX_VALUE); + results.add(Long.MAX_VALUE + ":2"); + }); + + fireTimers(scheduler); + + assertTrue(results.size() == 2); + assertEquals(results.get(0), key1 + ":1"); + assertEquals(results.get(1), key2 + ":2"); + } + + @Test + public void testRemoveTimer() { + ScheduledExecutorService service = mock(ScheduledExecutorService.class); + ScheduledFuture future = mock(ScheduledFuture.class); + when(future.cancel(anyBoolean())).thenReturn(true); + when(service.schedule((Runnable) anyObject(), anyLong(), anyObject())).thenAnswer(invocation -> future); + + SystemTimerScheduler scheduler = SystemTimerScheduler.create(service); + List results = new ArrayList<>(); + scheduler.setTimer("timer", 1, (key, collector, coordinator) -> { + results.add(key); + }); + + scheduler.deleteTimer("timer"); + + fireTimers(scheduler); + + assertTrue(results.isEmpty()); + verify(future, times(1)).cancel(anyBoolean()); + } + + @Test + public void testTimerListener() { + SystemTimerScheduler scheduler = SystemTimerScheduler.create(createExecutorService()); + List results = new ArrayList<>(); + scheduler.registerListener(() -> { + results.add("timer-listener"); + }); + + scheduler.setTimer("timer-listener", 1, (key, collector, coordinator) -> { + }); + + fireTimers(scheduler); + + assertTrue(results.size() == 1); + } +} diff --git a/samza-core/src/test/java/org/apache/samza/util/TestEmbeddedRateLimiter.java b/samza-core/src/test/java/org/apache/samza/util/TestEmbeddedRateLimiter.java deleted file mode 100644 index 1b3f687826..0000000000 --- a/samza-core/src/test/java/org/apache/samza/util/TestEmbeddedRateLimiter.java +++ /dev/null @@ -1,155 +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.Field; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; - -import org.apache.samza.SamzaException; -import org.apache.samza.config.Config; -import org.apache.samza.container.SamzaContainerContext; -import org.apache.samza.task.TaskContext; -import org.junit.Test; - -import junit.framework.Assert; - -import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - - -public class TestEmbeddedRateLimiter { - - final static private int TEST_INTERVAL = 200; // ms - final static private int TARGET_RATE = 4000; - final static private int NUMBER_OF_TASKS = 2; - final static private int TARGET_RATE_PER_TASK = TARGET_RATE / NUMBER_OF_TASKS; - final static private int INCREMENT = 2; - - @Test - public void testAcquire() { - RateLimiter rateLimiter = new EmbeddedRateLimiter(TARGET_RATE); - initRateLimiter(rateLimiter); - - int count = 0; - long start = System.currentTimeMillis(); - while (System.currentTimeMillis() - start < TEST_INTERVAL) { - rateLimiter.acquire(INCREMENT); - count += INCREMENT; - } - - long rate = count * 1000 / TEST_INTERVAL; - verifyRate(rate); - } - - @Test - public void testTryAcquire() { - RateLimiter rateLimiter = new EmbeddedRateLimiter(TARGET_RATE); - initRateLimiter(rateLimiter); - - boolean hasSeenZeros = false; - - int count = 0; - long start = System.currentTimeMillis(); - while (System.currentTimeMillis() - start < TEST_INTERVAL) { - int availableCredits = rateLimiter.tryAcquire(INCREMENT); - if (availableCredits <= 0) { - hasSeenZeros = true; - } else { - count += INCREMENT; - } - } - - long rate = count * 1000 / TEST_INTERVAL; - verifyRate(rate); - Assert.assertTrue(hasSeenZeros); - } - - @Test - public void testAcquireWithTimeout() { - RateLimiter rateLimiter = new EmbeddedRateLimiter(TARGET_RATE); - initRateLimiter(rateLimiter); - - boolean hasSeenZeros = false; - - int count = 0; - int callCount = 0; - long start = System.currentTimeMillis(); - while (System.currentTimeMillis() - start < TEST_INTERVAL) { - ++callCount; - int availableCredits = rateLimiter.acquire(INCREMENT, 20, MILLISECONDS); - if (availableCredits <= 0) { - hasSeenZeros = true; - } else { - count += INCREMENT; - } - } - - long rate = count * 1000 / TEST_INTERVAL; - verifyRate(rate); - Assert.assertTrue(Math.abs(callCount - TARGET_RATE_PER_TASK * TEST_INTERVAL / 1000 / INCREMENT) <= 2); - Assert.assertFalse(hasSeenZeros); - } - - @Test(expected = IllegalStateException.class) - public void testFailsWhenUninitialized() { - new EmbeddedRateLimiter(100).acquire(1); - } - - @Test(expected = IllegalArgumentException.class) - public void testFailsWhenUsingTags() { - RateLimiter rateLimiter = new EmbeddedRateLimiter(10); - initRateLimiter(rateLimiter); - Map tagToCredits = new HashMap<>(); - tagToCredits.put("red", 1); - tagToCredits.put("green", 1); - rateLimiter.acquire(tagToCredits); - } - - private void verifyRate(long rate) { - // As the actual rate would likely not be exactly the same as target rate, the calculation below - // verifies the actual rate is within 5% of the target rate per task - Assert.assertTrue(Math.abs(rate - TARGET_RATE_PER_TASK) <= TARGET_RATE_PER_TASK * 5 / 100); - } - - static void initRateLimiter(RateLimiter rateLimiter) { - Config config = mock(Config.class); - TaskContext taskContext = mock(TaskContext.class); - SamzaContainerContext containerContext = mockSamzaContainerContext(); - when(taskContext.getSamzaContainerContext()).thenReturn(containerContext); - rateLimiter.init(config, taskContext); - } - - static SamzaContainerContext mockSamzaContainerContext() { - try { - Collection taskNames = mock(Collection.class); - when(taskNames.size()).thenReturn(NUMBER_OF_TASKS); - SamzaContainerContext containerContext = mock(SamzaContainerContext.class); - Field taskNamesField = SamzaContainerContext.class.getDeclaredField("taskNames"); - taskNamesField.setAccessible(true); - taskNamesField.set(containerContext, taskNames); - taskNamesField.setAccessible(false); - return containerContext; - } catch (Exception ex) { - throw new SamzaException(ex); - } - } -} diff --git a/samza-core/src/test/java/org/apache/samza/util/TestEmbeddedTaggedRateLimiter.java b/samza-core/src/test/java/org/apache/samza/util/TestEmbeddedTaggedRateLimiter.java index a295d8fe34..05fa19aca3 100644 --- a/samza-core/src/test/java/org/apache/samza/util/TestEmbeddedTaggedRateLimiter.java +++ b/samza-core/src/test/java/org/apache/samza/util/TestEmbeddedTaggedRateLimiter.java @@ -18,13 +18,22 @@ */ package org.apache.samza.util; +import java.lang.reflect.Field; +import java.util.Collection; import java.util.HashMap; import java.util.Map; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.container.SamzaContainerContext; +import org.apache.samza.task.TaskContext; import org.junit.Assert; +import org.junit.Ignore; import org.junit.Test; import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class TestEmbeddedTaggedRateLimiter { @@ -34,41 +43,71 @@ public class TestEmbeddedTaggedRateLimiter { final static private int TARGET_RATE_RED = 1000; final static private int TARGET_RATE_PER_TASK_RED = TARGET_RATE_RED / NUMBER_OF_TASKS; final static private int TARGET_RATE_GREEN = 2000; - final static private int TARGET_RATE_PER_TASK_GREEN = TARGET_RATE_GREEN / NUMBER_OF_TASKS; final static private int INCREMENT = 2; + final static private int TARGET_RATE = 4000; + final static private int TARGET_RATE_PER_TASK = TARGET_RATE / NUMBER_OF_TASKS; + @Test + @Ignore("Flaky Test: Test fails in travis.") public void testAcquire() { - RateLimiter rateLimiter = createRateLimiter(); + RateLimiter rateLimiter = new EmbeddedTaggedRateLimiter(TARGET_RATE); + initRateLimiter(rateLimiter); - Map tagToCount = new HashMap<>(); - tagToCount.put("red", 0); - tagToCount.put("green", 0); + int count = 0; + long start = System.currentTimeMillis(); + while (System.currentTimeMillis() - start < TEST_INTERVAL) { + rateLimiter.acquire(INCREMENT); + count += INCREMENT; + } - Map tagToCredits = new HashMap<>(); - tagToCredits.put("red", INCREMENT); - tagToCredits.put("green", INCREMENT); + long rate = count * 1000 / TEST_INTERVAL; + verifyRate(rate); + } + + @Test + public void testAcquireWithTimeout() { + RateLimiter rateLimiter = new EmbeddedTaggedRateLimiter(TARGET_RATE); + initRateLimiter(rateLimiter); + + boolean hasSeenZeros = false; + int count = 0; + int callCount = 0; long start = System.currentTimeMillis(); while (System.currentTimeMillis() - start < TEST_INTERVAL) { - rateLimiter.acquire(tagToCredits); - tagToCount.put("red", tagToCount.get("red") + INCREMENT); - tagToCount.put("green", tagToCount.get("green") + INCREMENT); + ++callCount; + int availableCredits = rateLimiter.acquire(INCREMENT, 20, MILLISECONDS); + if (availableCredits <= 0) { + hasSeenZeros = true; + } else { + count += INCREMENT; + } } - { - long rate = tagToCount.get("red") * 1000 / TEST_INTERVAL; - verifyRate(rate, TARGET_RATE_PER_TASK_RED); - } { - // Note: due to blocking, green is capped at red's QPS - long rate = tagToCount.get("green") * 1000 / TEST_INTERVAL; - verifyRate(rate, TARGET_RATE_PER_TASK_RED); - } + long rate = count * 1000 / TEST_INTERVAL; + verifyRate(rate); + junit.framework.Assert.assertTrue(Math.abs(callCount - TARGET_RATE_PER_TASK * TEST_INTERVAL / 1000 / INCREMENT) <= 2); + junit.framework.Assert.assertFalse(hasSeenZeros); } - @Test - public void testTryAcquire() { + @Test(expected = IllegalStateException.class) + public void testFailsWhenUninitialized() { + new EmbeddedTaggedRateLimiter(100).acquire(1); + } + + @Test(expected = IllegalArgumentException.class) + public void testFailsWhenUsingTags() { + RateLimiter rateLimiter = new EmbeddedTaggedRateLimiter(10); + initRateLimiter(rateLimiter); + Map tagToCredits = new HashMap<>(); + tagToCredits.put("red", 1); + tagToCredits.put("green", 1); + rateLimiter.acquire(tagToCredits); + } + @Test + public void testAcquireTagged() { RateLimiter rateLimiter = createRateLimiter(); Map tagToCount = new HashMap<>(); @@ -81,22 +120,23 @@ public void testTryAcquire() { long start = System.currentTimeMillis(); while (System.currentTimeMillis() - start < TEST_INTERVAL) { - Map resultMap = rateLimiter.tryAcquire(tagToCredits); - tagToCount.put("red", tagToCount.get("red") + resultMap.get("red")); - tagToCount.put("green", tagToCount.get("green") + resultMap.get("green")); + rateLimiter.acquire(tagToCredits); + tagToCount.put("red", tagToCount.get("red") + INCREMENT); + tagToCount.put("green", tagToCount.get("green") + INCREMENT); } { long rate = tagToCount.get("red") * 1000 / TEST_INTERVAL; verifyRate(rate, TARGET_RATE_PER_TASK_RED); } { + // Note: due to blocking, green is capped at red's QPS long rate = tagToCount.get("green") * 1000 / TEST_INTERVAL; - verifyRate(rate, TARGET_RATE_PER_TASK_GREEN); + verifyRate(rate, TARGET_RATE_PER_TASK_RED); } } @Test - public void testAcquireWithTimeout() { + public void testAcquireWithTimeoutTagged() { RateLimiter rateLimiter = createRateLimiter(); @@ -126,7 +166,7 @@ public void testAcquireWithTimeout() { } @Test(expected = IllegalStateException.class) - public void testFailsWhenUninitialized() { + public void testFailsWhenUninitializedTagged() { Map tagToTargetRateMap = new HashMap<>(); tagToTargetRateMap.put("red", 1000); tagToTargetRateMap.put("green", 2000); @@ -139,14 +179,14 @@ public void testFailsWhenNotUsingTags() { tagToCredits.put("red", 1); tagToCredits.put("green", 1); RateLimiter rateLimiter = new EmbeddedTaggedRateLimiter(tagToCredits); - TestEmbeddedRateLimiter.initRateLimiter(rateLimiter); + initRateLimiter(rateLimiter); rateLimiter.acquire(1); } private void verifyRate(long rate, long targetRate) { // As the actual rate would likely not be exactly the same as target rate, the calculation below - // verifies the actual rate is within 5% of the target rate per task - Assert.assertTrue(Math.abs(rate - targetRate) <= targetRate * 5 / 100); + // verifies the actual rate is within 10% of the target rate per task + Assert.assertTrue(Math.abs(rate - targetRate) <= targetRate * 10 / 100); } private RateLimiter createRateLimiter() { @@ -154,8 +194,36 @@ private RateLimiter createRateLimiter() { tagToTargetRateMap.put("red", TARGET_RATE_RED); tagToTargetRateMap.put("green", TARGET_RATE_GREEN); RateLimiter rateLimiter = new EmbeddedTaggedRateLimiter(tagToTargetRateMap); - TestEmbeddedRateLimiter.initRateLimiter(rateLimiter); + initRateLimiter(rateLimiter); return rateLimiter; } + private void verifyRate(long rate) { + // As the actual rate would likely not be exactly the same as target rate, the calculation below + // verifies the actual rate is within 5% of the target rate per task + junit.framework.Assert.assertTrue(Math.abs(rate - TARGET_RATE_PER_TASK) <= TARGET_RATE_PER_TASK * 5 / 100); + } + + static void initRateLimiter(RateLimiter rateLimiter) { + Config config = mock(Config.class); + TaskContext taskContext = mock(TaskContext.class); + SamzaContainerContext containerContext = mockSamzaContainerContext(); + when(taskContext.getSamzaContainerContext()).thenReturn(containerContext); + rateLimiter.init(config, taskContext); + } + + static SamzaContainerContext mockSamzaContainerContext() { + try { + Collection taskNames = mock(Collection.class); + when(taskNames.size()).thenReturn(NUMBER_OF_TASKS); + SamzaContainerContext containerContext = mock(SamzaContainerContext.class); + Field taskNamesField = SamzaContainerContext.class.getDeclaredField("taskNames"); + taskNamesField.setAccessible(true); + taskNamesField.set(containerContext, taskNames); + taskNamesField.setAccessible(false); + return containerContext; + } catch (Exception ex) { + throw new SamzaException(ex); + } + } } diff --git a/samza-core/src/test/java/org/apache/samza/zk/TestZkUtils.java b/samza-core/src/test/java/org/apache/samza/zk/TestZkUtils.java index ec0494950b..1dfb414bce 100644 --- a/samza-core/src/test/java/org/apache/samza/zk/TestZkUtils.java +++ b/samza-core/src/test/java/org/apache/samza/zk/TestZkUtils.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.Map; import java.util.function.BooleanSupplier; +import com.google.common.collect.ImmutableList; import org.I0Itec.zkclient.IZkDataListener; import org.I0Itec.zkclient.ZkClient; import org.I0Itec.zkclient.ZkConnection; @@ -49,7 +50,6 @@ import org.junit.rules.ExpectedException; import org.mockito.Mockito; - public class TestZkUtils { private static EmbeddedZookeeper zkServer = null; private static final ZkKeyBuilder KEY_BUILDER = new ZkKeyBuilder("test"); @@ -121,6 +121,21 @@ public void testGetActiveProcessors() { Assert.assertEquals(1, zkUtils.getSortedActiveProcessorsZnodes().size()); } + @Test + public void testGetActiveProcessorIdShouldReturnEmptyForNonExistingZookeeperNodes() { + List processorsIDs = zkUtils.getActiveProcessorsIDs(ImmutableList.of("node1", "node2")); + + Assert.assertEquals(0, processorsIDs.size()); + } + + + @Test + public void testGetAllProcessorNodesShouldReturnEmptyForNonExistingZookeeperNodes() { + List processorsIDs = zkUtils.getAllProcessorNodes(); + + Assert.assertEquals(0, processorsIDs.size()); + } + @Test public void testZKProtocolVersion() { // first time connect, version should be set to ZkUtils.ZK_PROTOCOL_VERSION diff --git a/samza-core/src/test/scala/org/apache/samza/system/chooser/TestBootstrappingChooser.scala b/samza-core/src/test/scala/org/apache/samza/system/chooser/TestBootstrappingChooser.scala index e56206a4d4..02791bb698 100644 --- a/samza-core/src/test/scala/org/apache/samza/system/chooser/TestBootstrappingChooser.scala +++ b/samza-core/src/test/scala/org/apache/samza/system/chooser/TestBootstrappingChooser.scala @@ -36,9 +36,9 @@ import scala.collection.JavaConverters._ @RunWith(value = classOf[Parameterized]) class TestBootstrappingChooser(getChooser: (MessageChooser, Map[SystemStream, SystemStreamMetadata]) => MessageChooser) { - val envelope1 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream", new Partition(0)), null, null, 1); - val envelope2 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream1", new Partition(1)), null, null, 2); - val envelope3 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream1", new Partition(0)), null, null, 3); + val envelope1 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream", new Partition(0)), "120", null, 1); + val envelope2 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream1", new Partition(1)), "121", null, 2); + val envelope3 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream1", new Partition(0)), "122", null, 3); val envelope4 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream", new Partition(0)), "123", null, 4); /** diff --git a/samza-core/src/test/scala/org/apache/samza/system/chooser/TestDefaultChooser.scala b/samza-core/src/test/scala/org/apache/samza/system/chooser/TestDefaultChooser.scala index df5282cca2..e21bd9cb7f 100644 --- a/samza-core/src/test/scala/org/apache/samza/system/chooser/TestDefaultChooser.scala +++ b/samza-core/src/test/scala/org/apache/samza/system/chooser/TestDefaultChooser.scala @@ -32,13 +32,13 @@ import org.junit.Test import scala.collection.JavaConverters._ class TestDefaultChooser { - val envelope1 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream", new Partition(0)), null, null, 1); - val envelope2 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream1", new Partition(1)), null, null, 2); - val envelope3 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream2", new Partition(0)), null, null, 3); + val envelope1 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream", new Partition(0)), "120", null, 1); + val envelope2 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream1", new Partition(1)), "121", null, 2); + val envelope3 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream2", new Partition(0)), "122", null, 3); val envelope4 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream", new Partition(0)), "123", null, 4); - val envelope5 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream", new Partition(1)), null, null, 5); + val envelope5 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream", new Partition(1)), "320", null, 5); val envelope6 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream", new Partition(1)), "321", null, 6); - val envelope7 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream1", new Partition(0)), null, null, 7); + val envelope7 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream1", new Partition(0)), "653", null, 7); val envelope8 = new IncomingMessageEnvelope(new SystemStreamPartition("kafka", "stream3", new Partition(0)), "654", null, 8); @Test diff --git a/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/HdfsFileSystemAdapter.java b/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/HdfsFileSystemAdapter.java index bb7b3fadcd..07caaf7f09 100644 --- a/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/HdfsFileSystemAdapter.java +++ b/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/HdfsFileSystemAdapter.java @@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.samza.SamzaException; -import org.apache.samza.config.Config; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,7 +43,11 @@ public List getAllFiles(String streamName) { FileSystem fileSystem = streamPath.getFileSystem(new Configuration()); FileStatus[] fileStatuses = fileSystem.listStatus(streamPath); for (FileStatus fileStatus : fileStatuses) { - ret.add(new FileMetadata(fileStatus.getPath().toString(), fileStatus.getLen())); + if (!fileStatus.isDirectory()) { + ret.add(new FileMetadata(fileStatus.getPath().toString(), fileStatus.getLen())); + } else { + ret.addAll(getAllFiles(fileStatus.getPath().toString())); + } } } catch (IOException e) { LOG.error("Failed to get the list of files for " + streamName, e); diff --git a/samza-hdfs/src/test/java/org/apache/samza/system/hdfs/partitioner/TestHdfsFileSystemAdapter.java b/samza-hdfs/src/test/java/org/apache/samza/system/hdfs/partitioner/TestHdfsFileSystemAdapter.java index 0fb461fa07..a20e28566a 100644 --- a/samza-hdfs/src/test/java/org/apache/samza/system/hdfs/partitioner/TestHdfsFileSystemAdapter.java +++ b/samza-hdfs/src/test/java/org/apache/samza/system/hdfs/partitioner/TestHdfsFileSystemAdapter.java @@ -38,7 +38,7 @@ public void testGetAllFiles() FileSystemAdapter adapter = new HdfsFileSystemAdapter(); List result = adapter.getAllFiles(url.getPath()); - Assert.assertEquals(2, result.size()); + Assert.assertEquals(3, result.size()); } @Test diff --git a/samza-hdfs/src/test/resources/partitioner/subfolder/testfile002 b/samza-hdfs/src/test/resources/partitioner/subfolder/testfile002 new file mode 100644 index 0000000000..fe3e3b6af5 --- /dev/null +++ b/samza-hdfs/src/test/resources/partitioner/subfolder/testfile002 @@ -0,0 +1,16 @@ +censed 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. diff --git a/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemoryManager.java b/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemoryManager.java new file mode 100644 index 0000000000..4376ed39e0 --- /dev/null +++ b/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemoryManager.java @@ -0,0 +1,137 @@ +/* + * 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.inmemory; + +import java.io.IOException; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import org.apache.samza.Partition; +import org.apache.samza.system.IncomingMessageEnvelope; +import org.apache.samza.system.StreamSpec; +import org.apache.samza.system.SystemStreamMetadata; +import org.apache.samza.system.SystemStreamPartition; + + +/** + * + */ +public class InMemoryManager { + private final ConcurrentHashMap> bufferedMessages; + + public InMemoryManager() { + bufferedMessages = new ConcurrentHashMap<>(); + } + + public void register(SystemStreamPartition ssp, String offset) { + bufferedMessages.putIfAbsent(ssp, newSynchronizedLinkedList()); + } + + private LinkedList newSynchronizedLinkedList() { + return (LinkedList) Collections.synchronizedList(new LinkedList()); + } + + public void put(SystemStreamPartition ssp, Object message) { + bufferedMessages.computeIfAbsent(ssp, value -> new LinkedList<>()); + List messages = bufferedMessages.get(ssp); + int offset = messages.size(); + IncomingMessageEnvelope messageEnvelope = new IncomingMessageEnvelope(ssp, String.valueOf(offset), null, message); + bufferedMessages.get(ssp) + .addLast(messageEnvelope); + } + + public Map> poll(Map sspsToOffsets) { + return sspsToOffsets.entrySet() + .stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + entry -> poll(entry.getKey(), entry.getValue()))); + } + + private List poll(SystemStreamPartition ssp, String offset) { + int startingOffset = Integer.parseInt(offset); + List messageEnvelopesForSSP = bufferedMessages.getOrDefault(ssp, new LinkedList<>()); + + messageEnvelopesForSSP.add(IncomingMessageEnvelope.buildEndOfStreamEnvelope(ssp)); + + // we are at head and nothing to return + if (startingOffset >= messageEnvelopesForSSP.size()) { + return new LinkedList<>(); + } + return messageEnvelopesForSSP.subList(startingOffset, messageEnvelopesForSSP.size()); + } + + public boolean initializeStream(StreamSpec streamSpec, String serializedDataSet){ + try { + Set dataSet = InMemorySystemUtils.deserialize(serializedDataSet); + + int partitionCount = streamSpec.getPartitionCount(); + int partition = 0; + + for (Object data : dataSet) { + put(new SystemStreamPartition(streamSpec.toSystemStream(), new Partition(partition)), data); + partition = (partition + 1) % partitionCount; + } + } catch (Exception e) { + throw new RuntimeException(e); + } + + return true; + } + + public Map getSystemStreamMetadata(Set streamNames) { + Map>> result = + bufferedMessages.entrySet() + .stream() + .filter(map -> streamNames.contains(map.getKey().getStream())) + .collect(Collectors.groupingBy(entry -> entry.getKey().getStream(), + Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))); + + return result.entrySet() + .stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + entry -> constructSystemStreamMetadata(entry.getKey(), entry.getValue()))); + } + + private SystemStreamMetadata constructSystemStreamMetadata( + String systemName, + Map> sspToMessagesForSystem) { + + Map partitionMetadata = + sspToMessagesForSystem + .entrySet() + .stream() + .collect(Collectors.toMap(entry -> entry.getKey().getPartition(), entry -> { + String oldestOffset = "0"; + String newestOffset = String.valueOf(entry.getValue().size()); + String upcomingOffset = String.valueOf(entry.getValue().size() + 1); + + return new SystemStreamMetadata.SystemStreamPartitionMetadata(null, null, null); + + })); + + return new SystemStreamMetadata(systemName, partitionMetadata); + } +} diff --git a/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemAdmin.java b/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemAdmin.java new file mode 100644 index 0000000000..d504db19cd --- /dev/null +++ b/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemAdmin.java @@ -0,0 +1,154 @@ +/* + * 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.inmemory; + +import com.google.common.base.Preconditions; +import java.io.IOException; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.samza.config.Config; +import org.apache.samza.system.StreamSpec; +import org.apache.samza.system.StreamValidationException; +import org.apache.samza.system.SystemAdmin; +import org.apache.samza.system.SystemStreamMetadata; +import org.apache.samza.system.SystemStreamPartition; + + +/** + * + */ +public class InMemorySystemAdmin implements SystemAdmin { + private final InMemoryManager inMemoryManager; + private final InMemorySystemConfig inMemorySystemConfig; + + public InMemorySystemAdmin(InMemoryManager manager, Config config) { + inMemoryManager = manager; + inMemorySystemConfig = new InMemorySystemConfig(config); + } + + @Override + public void start() { + + } + + @Override + public void stop() { + + } + + /** + * Fetches the offsets for the messages immediately after the supplied offsets + * for a group of SystemStreamPartitions. + * + * @param offsets + * Map from SystemStreamPartition to current offsets. + * @return Map from SystemStreamPartition to offsets immediately after the + * current offsets. + */ + @Override + public Map getOffsetsAfter(Map offsets) { + return offsets.entrySet() + .stream() + .collect(Collectors.toMap(Map.Entry::getKey, entry -> { + String offset = entry.getValue(); + return String.valueOf(Integer.valueOf(offset) + 1); + })); + } + + /** + * Fetch metadata from a system for a set of streams. + * + * @param streamNames + * The streams to to fetch metadata for. + * @return A map from stream name to SystemStreamMetadata for each stream + * requested in the parameter set. + */ + @Override + public Map getSystemStreamMetadata(Set streamNames) { + return inMemoryManager.getSystemStreamMetadata(streamNames); + } + + /** + * Compare the two offsets. -1, 0, +1 means offset1 < offset2, + * offset1 == offset2 and offset1 > offset2 respectively. Return + * null if those two offsets are not comparable + * + * @param offset1 First offset for comparison. + * @param offset2 Second offset for comparison. + * @return -1 if offset1 < offset2; 0 if offset1 == offset2; 1 if offset1 > offset2. Null if not comparable + */ + @Override + public Integer offsetComparator(String offset1, String offset2) { +// Preconditions.checkNotNull(offset1); +// Preconditions.checkNotNull(offset2); +// +// int o1 = Integer.valueOf(offset1); +// int o2 = Integer.valueOf(offset2); +// +// return o1 - o2; + if (offset1 == null) { + return offset2 == null ? 0 : -1; + } else if (offset2 == null) { + return 1; + } + return offset1.compareTo(offset2); + } + + /** + * Create a stream described by the spec. + * + * @param streamSpec The spec, or blueprint from which the physical stream will be created on the system. + * @return {@code true} if the stream was actually created and not pre-existing. + * {@code false} if the stream was pre-existing. + * A RuntimeException will be thrown if creation fails. + */ + @Override + public boolean createStream(StreamSpec streamSpec) { + try { + return inMemoryManager.initializeStream(streamSpec, inMemorySystemConfig.getSerializedDataSet(streamSpec.getId())); + } catch (Exception e){ + throw new RuntimeException(e); + } + } + + /** + * Validates the stream described by the streamSpec on the system. + * A {@link StreamValidationException} should be thrown for any validation error. + * + * @param streamSpec The spec, or blueprint for the physical stream on the system. + * @throws StreamValidationException if validation fails. + */ + @Override + public void validateStream(StreamSpec streamSpec) throws StreamValidationException { + + } + + /** + * Clear the stream described by the spec. + * @param streamSpec The spec for the physical stream on the system. + * @return {@code true} if the stream was successfully cleared. + * {@code false} if clearing stream failed. + */ + @Override + public boolean clearStream(StreamSpec streamSpec) { + return false; + } +} diff --git a/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemConfig.java b/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemConfig.java new file mode 100644 index 0000000000..4abf7feb27 --- /dev/null +++ b/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemConfig.java @@ -0,0 +1,38 @@ +/* + * 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.inmemory; + +import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; + + +public class InMemorySystemConfig extends MapConfig { + // Serialized data set to initialize the consumer + private final String SERIALIZED_DATA_SET = "streams.%s.dataset"; + private final String SERIALIZED_DATA_SET_DEFAULT = ""; + + public InMemorySystemConfig(Config config) { + super(config); + } + + public String getSerializedDataSet(String streamId) { + return getOrDefault(String.format(SERIALIZED_DATA_SET, streamId), SERIALIZED_DATA_SET_DEFAULT); + } +} diff --git a/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemConsumer.java b/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemConsumer.java new file mode 100644 index 0000000000..fa4ebd6362 --- /dev/null +++ b/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemConsumer.java @@ -0,0 +1,143 @@ +/* + * 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.inmemory; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import org.apache.samza.system.IncomingMessageEnvelope; +import org.apache.samza.system.SystemConsumer; +import org.apache.samza.system.SystemStreamPartition; + + +/** + * + */ +public class InMemorySystemConsumer implements SystemConsumer { + private final InMemoryManager memoryManager; + private final Map sspToOffset; + + public InMemorySystemConsumer(InMemoryManager manager) { + memoryManager = manager; + sspToOffset = new ConcurrentHashMap<>(); + } + + /** + * Tells the SystemConsumer to connect to the underlying system, and prepare + * to begin serving messages when poll is invoked. + */ + @Override + public void start() { + + } + + /** + * Tells the SystemConsumer to close all connections, release all resource, + * and shut down everything. The SystemConsumer will not be used again after + * stop is called. + */ + @Override + public void stop() { + + } + + /** + * Register a SystemStreamPartition to this SystemConsumer. The SystemConsumer + * should try and read messages from all SystemStreamPartitions that are + * registered to it. SystemStreamPartitions should only be registered before + * start is called. + * @param systemStreamPartition + * The SystemStreamPartition object representing the Samza + * SystemStreamPartition to receive messages from. + * @param offset + * String representing the offset of the point in the stream to start + * reading messages from. This is an inclusive parameter; if "7" were + * specified, the first message for the system/stream/partition to be + * consumed and returned would be a message whose offset is "7". + * Note: For broadcast streams, different tasks may checkpoint the same ssp with different values. It + */ + @Override + public void register(SystemStreamPartition systemStreamPartition, String offset) { + sspToOffset.put(systemStreamPartition, "0"); + } + + /** + * Poll the SystemConsumer to get any available messages from the underlying + * system. + * + *

+ * If the underlying implementation does not take care to adhere to the + * timeout parameter, the SamzaContainer's performance will suffer + * drastically. Specifically, if poll blocks when it's not supposed to, it + * will block the entire main thread in SamzaContainer, and no messages will + * be processed while blocking is occurring. + *

+ * + * @param systemStreamPartitions + * A set of SystemStreamPartition to poll for new messages. If + * SystemConsumer has messages available for other registered + * SystemStreamPartitions, but they are not in the + * systemStreamPartitions set in a given poll invocation, they can't + * be returned. It is illegal to pass in SystemStreamPartitions that + * have not been registered with the SystemConsumer first. + * @param timeout + * If timeout < 0, poll will block unless all SystemStreamPartition + * are at "head" (the underlying system has been checked, and + * returned an empty set). If at head, an empty map is returned. If + * timeout >= 0, poll will return any messages that are currently + * available for any of the SystemStreamPartitions specified. If no + * new messages are available, it will wait up to timeout + * milliseconds for messages from any SystemStreamPartition to become + * available. It will return an empty map if the timeout is hit, and + * no new messages are available. + * @return A map from SystemStreamPartitions to any available + * IncomingMessageEnvelopes for the SystemStreamPartitions. If no + * messages are available for a SystemStreamPartition that was + * supplied in the polling set, the map will not contain a key for the + * SystemStreamPartition. Will return an empty map, not null, if no + * new messages are available for any SystemStreamPartitions in the + * input set. + * @throws InterruptedException + * Thrown when a blocking poll has been interrupted by another + * thread. + */ + @Override + public Map> poll( + Set systemStreamPartitions, long timeout) throws InterruptedException { + Map sspOffsetPairToFetch = sspToOffset.entrySet() + .stream() + .filter(entry -> systemStreamPartitions.contains(entry.getKey())) + .collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue())); + + Map> result = memoryManager.poll(sspOffsetPairToFetch); + + for(Map.Entry> sspToMessage : result.entrySet()) { + sspToOffset.computeIfPresent(sspToMessage.getKey(), (ssp, offset) -> { + int newOffset = Integer.parseInt(offset) + sspToMessage.getValue().size(); + return String.valueOf(newOffset); + }); + // absent should never be the case + } + + return result; + } +} diff --git a/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemFactory.java b/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemFactory.java new file mode 100644 index 0000000000..b2d8508e99 --- /dev/null +++ b/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemFactory.java @@ -0,0 +1,50 @@ +/* + * 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.inmemory; + +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; + + +/** + * + */ +public class InMemorySystemFactory implements SystemFactory { + private static final InMemoryManager MEMORY_MANAGER = new InMemoryManager(); + + @Override + public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) { + return new InMemorySystemConsumer(MEMORY_MANAGER); + } + + @Override + public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) { + return new InMemorySystemProducer(MEMORY_MANAGER); + } + + @Override + public SystemAdmin getAdmin(String systemName, Config config) { + return new InMemorySystemAdmin(MEMORY_MANAGER, config); + } +} diff --git a/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemProducer.java b/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemProducer.java new file mode 100644 index 0000000000..f7652f028c --- /dev/null +++ b/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemProducer.java @@ -0,0 +1,94 @@ +/* + * 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.inmemory; + +import java.util.Optional; +import org.apache.samza.Partition; +import org.apache.samza.system.OutgoingMessageEnvelope; +import org.apache.samza.system.SystemProducer; +import org.apache.samza.system.SystemStreamPartition; + + +/** + * + */ +public class InMemorySystemProducer implements SystemProducer { + private final InMemoryManager memoryManager; + + public InMemorySystemProducer(InMemoryManager manager) { + memoryManager = manager; + } + + /** + * Start the SystemProducer. After this method finishes it should be ready to accept messages received from the send method. + */ + @Override + public void start() { + + } + + /** + * Stop the SystemProducer. After this method finished, the system should have completed all necessary work, sent + * any remaining messages and will not receive any new calls to the send method. + */ + @Override + public void stop() { + + } + + /** + * Registers this producer to send messages from a specified Samza source, such as a StreamTask. + + * @param source String representing the source of the message. + */ + @Override + public void register(String source) { + + } + + /** + * Sends a specified message envelope from a specified Samza source. + + * @param source String representing the source of the message. + * @param envelope Aggregate object representing the serialized message to send from the source. + */ + @Override + public void send(String source, OutgoingMessageEnvelope envelope) { + Object message = envelope.getMessage(); + int partition = Optional.ofNullable(envelope.getPartitionKey()) + .map(Object::hashCode) + .orElse(0); + SystemStreamPartition ssp = new SystemStreamPartition(envelope.getSystemStream(), new Partition(partition)); + + memoryManager.put(ssp, message); + } + + /** + * If the SystemProducer buffers messages before sending them to its underlying system, it should flush those + * messages and leave no messages remaining to be sent. + * + + * @param source String representing the source of the message. + */ + @Override + public void flush(String source) { + // nothing to do + } +} diff --git a/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemUtils.java b/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemUtils.java new file mode 100644 index 0000000000..a3f9b8d148 --- /dev/null +++ b/samza-inmemory/src/main/java/org/apache/samza/system/inmemory/InMemorySystemUtils.java @@ -0,0 +1,107 @@ +/* + * 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.inmemory; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Base64; +import java.util.HashSet; +import java.util.Set; +import org.apache.samza.SamzaException; +import org.apache.samza.config.Config; +import org.apache.samza.serializers.Serde; +import org.apache.samza.serializers.SerdeFactory; +import org.codehaus.jackson.annotate.JsonCreator; +import org.codehaus.jackson.annotate.JsonProperty; +import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.type.TypeReference; + + +public class InMemorySystemUtils { + + public static Set deserialize(String serializedDataSet) throws IOException, ClassNotFoundException { + final byte[] bytes = Base64.getDecoder().decode(serializedDataSet); + final ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(bytes)); + @SuppressWarnings("unchecked") + Set object = new HashSet<>((ArrayList)ois.readObject()); + ois.close(); + // plugin base 64 decoder for now until deciding on the serialize format + return object; + } + + + public static class PageView implements Serializable { + @JsonProperty("pageKey") + final String pageKey; + @JsonProperty("memberId") + final int memberId; + + @JsonProperty("pageKey") + public String getPageKey() { + return pageKey; + } + + @JsonProperty("memberId") + public int getMemberId() { + return memberId; + } + + @JsonCreator + public PageView(@JsonProperty("pageKey") String pageKey, @JsonProperty("memberId") int memberId) { + this.pageKey = pageKey; + this.memberId = memberId; + } + } + + public static class PageViewJsonSerdeFactory implements SerdeFactory { + @Override + public Serde getSerde(String name, Config config) { + return new PageViewJsonSerde(); + } + } + + public static class PageViewJsonSerde implements Serde { + ObjectMapper mapper = new ObjectMapper(); + + @Override + public PageView fromBytes(byte[] bytes) { + try { + return mapper.readValue(new String(bytes, "UTF-8"), new TypeReference() { }); + } catch (Exception e) { + throw new SamzaException(e); + } + } + + @Override + public byte[] toBytes(PageView pv) { + try { + return mapper.writeValueAsString(pv).getBytes("UTF-8"); + } catch (Exception e) { + throw new SamzaException(e); + } + } + } + +} diff --git a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala index 2dd95695c5..b0901360c5 100644 --- a/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala +++ b/samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala @@ -20,6 +20,7 @@ package org.apache.samza.checkpoint.kafka import java.util.Collections +import java.util.concurrent.TimeUnit import com.google.common.base.Preconditions import org.apache.samza.checkpoint.{Checkpoint, CheckpointManager} @@ -53,8 +54,7 @@ class KafkaCheckpointManager(checkpointSpec: KafkaStreamSpec, checkpointMsgSerde: Serde[Checkpoint] = new CheckpointSerde, checkpointKeySerde: Serde[KafkaCheckpointLogKey] = new KafkaCheckpointLogKeySerde) extends CheckpointManager with Logging { - // Retry duration is approximately 83 minutes. - var MaxRetriesOnFailure = 50 + var MaxRetryDurationMs = TimeUnit.MINUTES.toMillis(15); info(s"Creating KafkaCheckpointManager for checkpointTopic:$checkpointTopic, systemName:$checkpointSystem " + s"validateCheckpoints:$validateCheckpoint") @@ -158,6 +158,7 @@ class KafkaCheckpointManager(checkpointSpec: KafkaStreamSpec, val envelope = new OutgoingMessageEnvelope(checkpointSsp, keyBytes, msgBytes) val retryBackoff: ExponentialSleepStrategy = new ExponentialSleepStrategy + val startTime = System.currentTimeMillis() retryBackoff.run( loop => { systemProducer.send(taskName.getTaskName, envelope) @@ -167,8 +168,8 @@ class KafkaCheckpointManager(checkpointSpec: KafkaStreamSpec, }, (exception, loop) => { - if (loop.sleepCount >= MaxRetriesOnFailure) { - error(s"Exhausted $MaxRetriesOnFailure retries when writing checkpoint: $checkpoint for task: $taskName.") + if ((System.currentTimeMillis() - startTime) >= MaxRetryDurationMs) { + error(s"Exhausted $MaxRetryDurationMs milliseconds when writing checkpoint: $checkpoint for task: $taskName.") throw new SamzaException(s"Exception when writing checkpoint: $checkpoint for task: $taskName.", exception) } else { warn(s"Retrying failed checkpoint write to key: $key, checkpoint: $checkpoint for task: $taskName", exception) diff --git a/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala b/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala index 5586a1a6a3..03b0d2c07a 100644 --- a/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala +++ b/samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala @@ -115,7 +115,7 @@ class TestKafkaCheckpointManager extends KafkaServerTestHarness { val props = new org.apache.samza.config.KafkaConfig(config).getCheckpointTopicProperties() val spec = new KafkaStreamSpec("id", checkpointTopic, checkpointSystemName, 1, 1, false, props) val checkPointManager = new KafkaCheckpointManager(spec, new MockSystemFactory, false, config, new NoOpMetricsRegistry) - checkPointManager.MaxRetriesOnFailure = 1 + checkPointManager.MaxRetryDurationMs = 1 checkPointManager.register(taskName) checkPointManager.start diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java index 4af0f1dc78..b494eba9b0 100644 --- a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java +++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableProvider.java @@ -25,18 +25,26 @@ import org.apache.samza.config.JavaTableConfig; import org.apache.samza.config.MapConfig; import org.apache.samza.config.StorageConfig; -import org.apache.samza.storage.StorageEngine; -import org.apache.samza.table.LocalStoreBackedTableProvider; +import org.apache.samza.container.SamzaContainerContext; +import org.apache.samza.table.ReadableTable; import org.apache.samza.table.Table; +import org.apache.samza.table.TableProvider; import org.apache.samza.table.TableSpec; +import org.apache.samza.task.TaskContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.base.Preconditions; + /** - * Base class for tables backed by Samza stores, see {@link LocalStoreBackedTableProvider}. + * Base class for tables backed by Samza local stores. The backing stores are + * injected during initialization of the table. Since the lifecycle + * of the underlying stores are already managed by Samza container, + * the table provider will not manage the lifecycle of the backing + * stores. */ -abstract public class BaseLocalStoreBackedTableProvider implements LocalStoreBackedTableProvider { +abstract public class BaseLocalStoreBackedTableProvider implements TableProvider { protected final Logger logger = LoggerFactory.getLogger(getClass()); @@ -44,13 +52,28 @@ abstract public class BaseLocalStoreBackedTableProvider implements LocalStoreBac protected KeyValueStore kvStore; + protected SamzaContainerContext containerContext; + + protected TaskContext taskContext; + public BaseLocalStoreBackedTableProvider(TableSpec tableSpec) { this.tableSpec = tableSpec; } @Override - public void init(StorageEngine store) { - kvStore = (KeyValueStore) store; + public void init(SamzaContainerContext containerContext, TaskContext taskContext) { + this.containerContext = containerContext; + this.taskContext = taskContext; + + Preconditions.checkNotNull(this.taskContext, "Must specify task context for local tables."); + + kvStore = (KeyValueStore) taskContext.getStore(tableSpec.getId()); + + if (kvStore == null) { + throw new SamzaException(String.format( + "Backing store for table %s was not injected by SamzaContainer", tableSpec.getId())); + } + logger.info("Initialized backing store for table " + tableSpec.getId()); } @@ -59,17 +82,9 @@ public Table getTable() { if (kvStore == null) { throw new SamzaException("Store not initialized for table " + tableSpec.getId()); } - return new LocalStoreBackedReadWriteTable(kvStore); - } - - @Override - public void start() { - logger.info("Starting table provider for table " + tableSpec.getId()); - } - - @Override - public void stop() { - logger.info("Stopping table provider for table " + tableSpec.getId()); + ReadableTable table = new LocalStoreBackedReadWriteTable(tableSpec.getId(), kvStore); + table.init(containerContext, taskContext); + return table; } protected Map generateCommonStoreConfig(Map config) { @@ -89,4 +104,9 @@ protected Map generateCommonStoreConfig(Map conf return storeConfig; } + + @Override + public void close() { + logger.info("Shutting down table provider for table " + tableSpec.getId()); + } } diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadWriteTable.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadWriteTable.java index 3149c86eaf..4037f60dec 100644 --- a/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadWriteTable.java +++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadWriteTable.java @@ -36,8 +36,8 @@ public class LocalStoreBackedReadWriteTable extends LocalStoreBackedReadab * Constructs an instance of {@link LocalStoreBackedReadWriteTable} * @param kvStore the backing store */ - public LocalStoreBackedReadWriteTable(KeyValueStore kvStore) { - super(kvStore); + public LocalStoreBackedReadWriteTable(String tableId, KeyValueStore kvStore) { + super(tableId, kvStore); } @Override diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadableTable.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadableTable.java index fead086348..5ff58ab7d2 100644 --- a/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadableTable.java +++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadableTable.java @@ -24,6 +24,8 @@ import org.apache.samza.table.ReadableTable; +import com.google.common.base.Preconditions; + /** * A store backed readable table @@ -34,12 +36,16 @@ public class LocalStoreBackedReadableTable implements ReadableTable { protected KeyValueStore kvStore; + protected String tableId; /** * Constructs an instance of {@link LocalStoreBackedReadableTable} * @param kvStore the backing store */ - public LocalStoreBackedReadableTable(KeyValueStore kvStore) { + public LocalStoreBackedReadableTable(String tableId, KeyValueStore kvStore) { + Preconditions.checkArgument(tableId != null & !tableId.isEmpty() , "invalid tableId"); + Preconditions.checkNotNull(kvStore, "null KeyValueStore"); + this.tableId = tableId; this.kvStore = kvStore; } diff --git a/samza-kv/src/test/java/org/apache/samza/storage/kv/TestLocalBaseStoreBackedTableProvider.java b/samza-kv/src/test/java/org/apache/samza/storage/kv/TestLocalBaseStoreBackedTableProvider.java index 9c95637834..d30c18f4ae 100644 --- a/samza-kv/src/test/java/org/apache/samza/storage/kv/TestLocalBaseStoreBackedTableProvider.java +++ b/samza-kv/src/test/java/org/apache/samza/storage/kv/TestLocalBaseStoreBackedTableProvider.java @@ -27,11 +27,13 @@ import org.apache.samza.config.StorageConfig; import org.apache.samza.storage.StorageEngine; import org.apache.samza.table.TableSpec; +import org.apache.samza.task.TaskContext; import org.junit.Before; import org.junit.Test; import junit.framework.Assert; +import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -55,7 +57,9 @@ public Map generateConfig(Map config) { @Test public void testInit() { StorageEngine store = mock(KeyValueStorageEngine.class); - tableProvider.init(store); + TaskContext taskContext = mock(TaskContext.class); + when(taskContext.getStore(any())).thenReturn(store); + tableProvider.init(null, taskContext); Assert.assertNotNull(tableProvider.getTable()); } diff --git a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java index d432be7eb2..83928e1502 100644 --- a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java +++ b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java @@ -94,14 +94,14 @@ public static Config computeSamzaConfigs(Boolean localRunner, Config config) { newConfig.putAll(outputSystemStreamConfig.getConfig()); } + newConfig.putAll(config); + if (localRunner) { newConfig.put(RUNNER_CONFIG, LocalApplicationRunner.class.getName()); } else { newConfig.put(RUNNER_CONFIG, RemoteApplicationRunner.class.getName()); } - newConfig.putAll(config); - LOG.info("New Samza configs: " + newConfig); return new MapConfig(newConfig); } diff --git a/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlApplicationRunner.java b/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlApplicationRunner.java new file mode 100644 index 0000000000..e42b55d77c --- /dev/null +++ b/samza-sql/src/test/java/org/apache/samza/sql/TestSamzaSqlApplicationRunner.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.sql; + +import java.util.Map; + +import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; +import org.apache.samza.runtime.LocalApplicationRunner; +import org.apache.samza.runtime.RemoteApplicationRunner; +import org.apache.samza.sql.runner.SamzaSqlApplicationRunner; +import org.apache.samza.sql.testutil.SamzaSqlTestConfig; +import org.junit.Assert; + +import org.apache.samza.sql.runner.SamzaSqlApplicationConfig; +import org.junit.Test; + + +public class TestSamzaSqlApplicationRunner { + + @Test + public void testComputeSamzaConfigs() { + Map configs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10); + String sql1 = "Insert into testavro.outputTopic select id, MyTest(id) as long_value from testavro.SIMPLE1"; + configs.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql1); + configs.put(SamzaSqlApplicationRunner.RUNNER_CONFIG, SamzaSqlApplicationRunner.class.getName()); + MapConfig samzaConfig = new MapConfig(configs); + Config newConfigs = SamzaSqlApplicationRunner.computeSamzaConfigs(true, samzaConfig); + Assert.assertEquals(newConfigs.get(SamzaSqlApplicationRunner.RUNNER_CONFIG), LocalApplicationRunner.class.getName()); + // Check whether three new configs added. + Assert.assertEquals(newConfigs.size(), configs.size() + 3); + + newConfigs = SamzaSqlApplicationRunner.computeSamzaConfigs(false, samzaConfig); + Assert.assertEquals(newConfigs.get(SamzaSqlApplicationRunner.RUNNER_CONFIG), RemoteApplicationRunner.class.getName()); + + // Check whether three new configs added. + Assert.assertEquals(newConfigs.size(), configs.size() + 3); + } +} diff --git a/samza-test/src/test/java/org/apache/samza/test/util/StreamAssert.java b/samza-test/src/main/java/org/apache/samza/system/framework/utils/StreamAssert.java similarity index 52% rename from samza-test/src/test/java/org/apache/samza/test/util/StreamAssert.java rename to samza-test/src/main/java/org/apache/samza/system/framework/utils/StreamAssert.java index 8a46db04ec..81432dba5f 100644 --- a/samza-test/src/test/java/org/apache/samza/test/util/StreamAssert.java +++ b/samza-test/src/main/java/org/apache/samza/system/framework/utils/StreamAssert.java @@ -17,16 +17,25 @@ * under the License. */ -package org.apache.samza.test.util; +package org.apache.samza.system.framework.utils; +import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; +import java.util.HashMap; +import java.util.stream.Collectors; import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; import org.apache.samza.operators.MessageStream; import org.apache.samza.operators.functions.SinkFunction; import org.apache.samza.serializers.KVSerde; import org.apache.samza.serializers.Serde; import org.apache.samza.serializers.StringSerde; +import org.apache.samza.system.EndOfStreamMessage; +import org.apache.samza.system.IncomingMessageEnvelope; +import org.apache.samza.system.SystemStreamMetadata; import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.system.inmemory.InMemorySystemConsumer; +import org.apache.samza.system.inmemory.InMemorySystemFactory; import org.apache.samza.task.MessageCollector; import org.apache.samza.task.TaskContext; import org.apache.samza.task.TaskCoordinator; @@ -43,13 +52,16 @@ import java.util.TimerTask; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; +import org.hamcrest.collection.IsIterableContainingInAnyOrder; +import org.hamcrest.collection.IsIterableContainingInOrder; +import static org.hamcrest.collection.IsCollectionWithSize.*; import static org.junit.Assert.assertThat; /** * An assertion on the content of a {@link MessageStream}. * - *

Example: {@code + *

Example: {@code
  * MessageStream stream = streamGraph.getInputStream("input", serde).map(some_function)...;
  * ...
  * StreamAssert.that(id, stream, stringSerde).containsInAnyOrder(Arrays.asList("a", "b", "c"));
@@ -64,6 +76,9 @@ public class StreamAssert {
   private final MessageStream messageStream;
   private final Serde serde;
   private boolean checkEachTask = false;
+  private String systemStream;
+  private String systemName;
+
 
   public static  StreamAssert that(String id, MessageStream messageStream, Serde serde) {
     return new StreamAssert<>(id, messageStream, serde);
@@ -164,4 +179,93 @@ private void check() {
       }
     }
   }
+
+
+  private StreamAssert(String streamId) {
+    Preconditions.checkState(streamId.indexOf(".") > 0 && streamId.indexOf(".") < streamId.length() - 1);
+    this.systemStream = streamId.substring(streamId.indexOf(".") + 1);
+    this.systemName = streamId.substring(0, streamId.indexOf("."));
+    this.id = streamId;
+    this.serde = null;
+    this.messageStream = null;
+  }
+
+  public static  StreamAssert that(String streamId) {
+    return new StreamAssert<>(streamId);
+  }
+
+  private List consume() throws InterruptedException {
+    InMemorySystemFactory factory = new InMemorySystemFactory();
+    Set ssps = new HashSet<>();
+    Set streamNames = new HashSet<>();
+    streamNames.add(systemStream);
+    Map metadata =
+        factory.getAdmin(systemName, new MapConfig()).getSystemStreamMetadata(streamNames);
+    InMemorySystemConsumer consumer =
+        (InMemorySystemConsumer) factory.getConsumer(systemName, new MapConfig(new HashMap<>()), null);
+    metadata.get(systemStream).getSystemStreamPartitionMetadata().keySet().forEach(partition -> {
+      SystemStreamPartition temp = new SystemStreamPartition(systemName, systemStream, partition);
+      ssps.add(temp);
+      consumer.register(temp, "0");
+    });
+    Map> output = consumer.poll(ssps, 10);
+    return output.values()
+        .stream()
+        .flatMap(List::stream)
+        .map(e -> (M) e.getMessage())
+        .filter(e -> !(e instanceof EndOfStreamMessage))
+        .collect(Collectors.toList());
+  }
+
+  private Map> consumePartitions() throws InterruptedException {
+    InMemorySystemFactory factory = new InMemorySystemFactory();
+    Set ssps = new HashSet<>();
+    Set streamNames = new HashSet<>();
+    streamNames.add(systemStream);
+    Map metadata =
+        factory.getAdmin(systemName, new MapConfig()).getSystemStreamMetadata(streamNames);
+    InMemorySystemConsumer consumer =
+        (InMemorySystemConsumer) factory.getConsumer(systemName, new MapConfig(new HashMap<>()), null);
+    metadata.get(systemStream).getSystemStreamPartitionMetadata().keySet().forEach(partition -> {
+      SystemStreamPartition temp = new SystemStreamPartition(systemName, systemStream, partition);
+      ssps.add(temp);
+      consumer.register(temp, "0");
+    });
+    Map> output = consumer.poll(ssps, 10);
+    Map map = output.entrySet()
+        .stream()
+        .collect(Collectors.toMap(entry -> entry.getKey().getPartition().getPartitionId(),
+            entry -> entry.getValue()
+            .stream()
+            .map(e -> (M) e.getMessage())
+            .filter(e -> !(e instanceof EndOfStreamMessage))
+            .collect(Collectors.toList())));
+    return map;
+  }
+
+  public void containsInAnyOrder(List expected) throws InterruptedException {
+    assertThat(consume(), IsIterableContainingInAnyOrder.containsInAnyOrder(expected.toArray()));
+  }
+
+  public void containsInAnyOrder(Map expected) throws InterruptedException {
+    Map> actual = consumePartitions();
+    for(Integer paritionId: expected.keySet()){
+      assertThat(actual.get(paritionId), IsIterableContainingInAnyOrder.containsInAnyOrder(expected.get(paritionId).toArray()));
+    }
+  }
+
+  public void contains(Map expected) throws InterruptedException {
+    Map> actual = consumePartitions();
+    for(Integer paritionId: expected.keySet()){
+      assertThat(actual.get(paritionId), IsIterableContainingInOrder.contains(expected.get(paritionId).toArray()));
+    }
+  }
+
+  public void contains(List expected) throws InterruptedException {
+    assertThat(consume(), IsIterableContainingInOrder.contains(expected.toArray()));
+  }
+
+  public void size(Integer size) throws InterruptedException {
+    assertThat(consume(), hasSize(size));
+  }
 }
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/InMemoryCollectionStreamSystem.java b/samza-test/src/main/java/org/apache/samza/test/framework/InMemoryCollectionStreamSystem.java
new file mode 100644
index 0000000000..a091b3d24a
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/InMemoryCollectionStreamSystem.java
@@ -0,0 +1,107 @@
+package org.apache.samza.test.framework;
+
+import com.google.common.base.Preconditions;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import javafx.util.Pair;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.system.EndOfStreamMessage;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemProducer;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.system.inmemory.InMemorySystemConsumer;
+import org.apache.samza.system.inmemory.InMemorySystemFactory;
+
+
+public class InMemoryCollectionStreamSystem {
+  private static final String SYSTEM_FACTORY = "systems.%s.samza.factory";
+  private static final String SYSTEM_OFFSET = "systems.%s.default.stream.samza.offset.default";
+  private static final String STREAM_TO_SYSTEM = "streams.%s.samza.system";
+
+  // Name of the System
+  private String name;
+
+  // Maintain the global job config
+  private Map systemConfigs;
+
+  // InMemorySystemFactory
+  private InMemorySystemFactory factoryTest;
+
+  private InMemoryCollectionStreamSystem(String name){
+    this.name = name;
+    factoryTest = new InMemorySystemFactory();
+    systemConfigs = new HashMap();
+    // System Factory InMemory System
+    systemConfigs.putIfAbsent(String.format(SYSTEM_FACTORY,name), InMemorySystemFactory.class.getName());
+    // Consume from the oldest for all streams in the system
+    systemConfigs.putIfAbsent(String.format(SYSTEM_OFFSET,name), "oldest");
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Map getSystemConfigs(){
+    return systemConfigs;
+  }
+
+  public static InMemoryCollectionStreamSystem create(String name){
+    Preconditions.checkState(name !=null);
+    return new InMemoryCollectionStreamSystem(name);
+  }
+
+  public InMemoryCollectionStreamSystem addOutput(String steamId) {
+    Preconditions.checkNotNull(steamId);
+    systemConfigs.put(String.format(STREAM_TO_SYSTEM, steamId), name);
+    return this;
+  }
+
+  public  InMemoryCollectionStreamSystem addInput(String systemName, String streamId, Map partitions) {
+    Preconditions.checkState(streamId != null);
+    Preconditions.checkState(name != null);
+
+    SystemProducer producer = new InMemorySystemFactory().getProducer(systemName, new MapConfig(systemConfigs), null);
+    partitions.forEach((partitionId, partition) -> {
+      partition.forEach(e -> {
+        Object key = e instanceof Pair ? ((Pair) e).getKey() : null;
+        Object value = e instanceof Pair ? ((Pair) e).getValue() : e;
+        producer.send(systemName,
+            new OutgoingMessageEnvelope(new SystemStream(systemName, streamId), Integer.valueOf(partitionId), key, value));
+
+      });
+    });
+    return this;
+  }
+
+  public  List getStreamState(String streamId) throws InterruptedException {
+
+    Set ssps = new HashSet<>();
+    Set streamNames = new HashSet<>();
+    streamNames.add(streamId);
+    Map metadata =
+        factoryTest.getAdmin(name, new MapConfig()).getSystemStreamMetadata(streamNames);
+    InMemorySystemConsumer consumer =
+        (InMemorySystemConsumer) factoryTest.getConsumer(name, new MapConfig(new HashMap<>()), null);
+    metadata.get(streamId).getSystemStreamPartitionMetadata().keySet().forEach(partition -> {
+      SystemStreamPartition temp = new SystemStreamPartition(name, streamId, partition);
+      ssps.add(temp);
+      consumer.register(temp, "0");
+    });
+    Map> output = consumer.poll(ssps, 10);
+    return output.values()
+        .stream()
+        .flatMap(List::stream)
+        .map(e -> (T) e.getMessage())
+        .filter(e -> !(e instanceof EndOfStreamMessage))
+        .collect(Collectors.toList());
+  }
+
+}
+
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/Mode.java b/samza-test/src/main/java/org/apache/samza/test/framework/Mode.java
new file mode 100644
index 0000000000..fb66892ebb
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/Mode.java
@@ -0,0 +1,6 @@
+package org.apache.samza.test.framework;
+
+public enum Mode
+{
+  SINGLE_CONTAINER, MULTI_CONTAINER;
+}
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/TestApplication.java b/samza-test/src/main/java/org/apache/samza/test/framework/TestApplication.java
new file mode 100644
index 0000000000..d76e96c0f9
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/TestApplication.java
@@ -0,0 +1,69 @@
+package org.apache.samza.test.framework;
+
+import com.google.common.base.Preconditions;
+import java.util.HashMap;
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.JobCoordinatorConfig;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.config.TaskConfig;
+import org.apache.samza.container.grouper.task.SingleContainerGrouperFactory;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.standalone.PassthroughCoordinationUtilsFactory;
+import org.apache.samza.standalone.PassthroughJobCoordinatorFactory;
+import org.apache.samza.system.inmemory.InMemorySystemFactory;
+import org.apache.samza.task.StreamTask;
+import org.apache.samza.test.framework.stream.CollectionStream;
+import org.apache.samza.test.framework.stream.EventStream;
+import org.apache.samza.test.framework.stream.FileStream;
+
+
+public class TestApplication {
+  private StreamApplication application;
+  public HashMap configs;
+  private InMemorySystemFactory factoryTest;
+  private Mode mode;
+  public static final String SYSTEM_NAME = "test-samza-application";
+  public static final String JOB_NAME = "test-application";
+
+  private TestApplication( HashMap configs, Mode mode) {
+    Preconditions.checkNotNull(configs);
+    Preconditions.checkNotNull(mode);
+
+    this.configs = configs;
+    this.mode = mode;
+    factoryTest = new InMemorySystemFactory();
+
+    // JOB Specific Config
+    configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), SYSTEM_NAME);
+    configs.put(JobConfig.JOB_NAME(), JOB_NAME);
+
+    if(mode.equals(mode.SINGLE_CONTAINER)) {
+      configs.putIfAbsent(JobConfig.PROCESSOR_ID(), "1");
+      configs.putIfAbsent(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, PassthroughCoordinationUtilsFactory.class.getName());
+      configs.putIfAbsent(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, PassthroughJobCoordinatorFactory.class.getName());
+      configs.putIfAbsent(TaskConfig.GROUPER_FACTORY(), SingleContainerGrouperFactory.class.getName());
+    } else if(mode.equals(mode.MULTI_CONTAINER)){
+
+    }
+
+    // InMemory System
+    configs.put("systems." + SYSTEM_NAME + ".samza.factory", InMemorySystemFactory.class.getName()); // system factory
+
+    /**
+     * Configration of Stream Graph, stream app
+     */
+  }
+
+  public static TestApplication create(HashMap configs, Mode mode) {
+    return new TestApplication(configs, mode);
+  }
+  public void run() {
+    final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs));
+    runner.run(application);
+    runner.waitForFinish();
+  };
+}
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java
new file mode 100644
index 0000000000..0c51c5caa9
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java
@@ -0,0 +1,143 @@
+package org.apache.samza.test.framework;
+
+import com.google.common.base.Preconditions;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.JobCoordinatorConfig;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.config.TaskConfig;
+import org.apache.samza.container.grouper.task.SingleContainerGrouperFactory;
+import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.standalone.PassthroughCoordinationUtilsFactory;
+import org.apache.samza.standalone.PassthroughJobCoordinatorFactory;
+import org.apache.samza.task.AsyncStreamTask;
+import org.apache.samza.task.StreamTask;
+import org.apache.samza.test.framework.stream.CollectionStream;
+
+
+public class TestRunner {
+
+  // Maintain the global job config
+  private static Map configs;
+
+  private static Map systems;
+
+  // Default job name
+  private static final String JOB_NAME = "test-samza";
+
+  // Either StreamTask or AsyncStreamTask exist
+  private StreamTask syncTask;
+  private AsyncStreamTask asyncTask;
+  private StreamApplication app;
+
+  // Mode defines single or multi container
+  private Mode mode;
+
+
+  private TestRunner(){
+    this.configs = new HashMap<>();
+    this.mode = Mode.SINGLE_CONTAINER;
+    this.systems = new HashMap();
+
+    // JOB Specific Config
+    configs.put(JobConfig.JOB_NAME(), JOB_NAME);
+
+    // Default Single Container configs
+    configs.putIfAbsent(JobConfig.PROCESSOR_ID(), "1");
+    configs.putIfAbsent(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, PassthroughCoordinationUtilsFactory.class.getName());
+    configs.putIfAbsent(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, PassthroughJobCoordinatorFactory.class.getName());
+    configs.putIfAbsent(TaskConfig.GROUPER_FACTORY(), SingleContainerGrouperFactory.class.getName());
+
+  }
+
+  private TestRunner(StreamTask task) {
+    this();
+    Preconditions.checkNotNull(task);
+    this.syncTask = task;
+  }
+
+  private TestRunner(AsyncStreamTask task) {
+    this();
+    Preconditions.checkNotNull(task);
+    this.asyncTask = task;
+  }
+
+  private TestRunner(StreamApplication app) {
+    this();
+    Preconditions.checkNotNull(app);
+    this.app = app;
+  }
+
+  public static TestRunner of(StreamTask task) {
+    return new TestRunner(task);
+  }
+
+  public static TestRunner of(AsyncStreamTask task) {
+    return new TestRunner(task);
+  }
+
+  public static TestRunner of(StreamApplication app) {
+    return new TestRunner(app);
+  }
+
+  public static InMemoryCollectionStreamSystem getOrIntializeInMemoryCollectionStreamSystem(String systemName) {
+    Preconditions.checkState(systems != null);
+    if(!systems.containsKey(systemName)) {
+      InMemoryCollectionStreamSystem sys = InMemoryCollectionStreamSystem.create(systemName);
+      systems.put(systemName, sys);
+      configs.putAll(sys.getSystemConfigs());
+    }
+    return (InMemoryCollectionStreamSystem)systems.get(systemName);
+  }
+
+  public TestRunner addOverrideConfigs(Map config) {
+    Preconditions.checkNotNull(config);
+    this.configs.putAll(config);
+    return this;
+  }
+
+  public TestRunner setContainerMode(Mode mode) {
+    Preconditions.checkNotNull(mode);
+    if(mode.equals(Mode.MULTI_CONTAINER)){ // zk based config
+      // zk based config
+    }
+    return this;
+  }
+
+  public TestRunner addInputStream(CollectionStream stream) {
+    Preconditions.checkNotNull(stream);
+    InMemoryCollectionStreamSystem system = getOrIntializeInMemoryCollectionStreamSystem(stream.getSystemName());
+    system.addInput(stream.getSystemName(), stream.getStreamId(),stream.getInitPartitions());
+    if(configs.containsKey(TaskConfig.INPUT_STREAMS()))
+      configs.put(TaskConfig.INPUT_STREAMS(), configs.get(TaskConfig.INPUT_STREAMS()).concat(","+stream.getSystemName()+"."+stream.getStreamId()));
+    stream.getStreamConfig().forEach((key,val) -> {
+      configs.putIfAbsent((String)key, (String) val);
+    });
+    return this;
+  }
+
+  public TestRunner addOutputStream(CollectionStream stream) {
+    Preconditions.checkNotNull(stream);
+    configs.putAll(stream.getStreamConfig());
+    return this;
+  }
+
+  public void run() throws Exception {
+    final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs));
+    if (syncTask != null && asyncTask == null && app == null) {
+      runner.runSyncTask(syncTask);
+      runner.waitForFinish();
+    } else if(asyncTask != null && syncTask == null && app == null) {
+      runner.runAsyncTask(asyncTask);
+      runner.waitForFinish();
+    } else if(asyncTask == null && syncTask == null && app != null) {
+      runner.run(app);
+      runner.waitForFinish();
+    } else {
+      throw new Exception("Test should use either one config async, application or sync");
+    }
+  }
+
+}
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/examples/SampleAsyncTaskTest.java b/samza-test/src/main/java/org/apache/samza/test/framework/examples/SampleAsyncTaskTest.java
new file mode 100644
index 0000000000..9397cd472a
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/examples/SampleAsyncTaskTest.java
@@ -0,0 +1,92 @@
+package org.apache.samza.test.framework.examples;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Random;
+import org.apache.samza.config.Config;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.framework.utils.StreamAssert;
+import org.apache.samza.task.AsyncStreamTask;
+import org.apache.samza.task.ClosableTask;
+import org.apache.samza.task.InitableTask;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.TaskCallback;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.task.TaskCoordinator;
+import org.apache.samza.test.framework.Mode;
+import org.junit.Test;
+import scala.Int;
+
+
+public class SampleAsyncTaskTest {
+
+  public static class AsyncRestTask implements AsyncStreamTask, InitableTask, ClosableTask {
+
+    @Override
+    public void init(Config config, TaskContext taskContext) throws Exception {
+      // Your initialization of web client code goes here
+    }
+
+    @Override
+    public void processAsync(IncomingMessageEnvelope envelope, MessageCollector collector,
+        TaskCoordinator coordinator, final TaskCallback callback) {
+        // Mimic a random callback delay ans send message
+        RestCall call = new RestCall(envelope, collector, callback);
+        call.start();
+    }
+
+    @Override
+    public void close() throws Exception {
+      // Close your client
+    }
+  }
+
+  @Test
+  public void testAsyncTaskWithConcurrency() throws Exception{
+    List input = Arrays.asList(1,2,3,4,5);
+    List output = Arrays.asList(10,20,30,40,50);
+
+//    // Run the test framework
+//    TestRunner
+//        .of(new AsyncRestTask())
+//        .setTaskCallBackTimeoutMS(200)
+//        .addInputStream(CollectionStream.of("test.input", input))
+//        .addOutputStream(CollectionStream.empty("test.Output"))
+//        .run();
+
+
+    StreamAssert.that("test.Output").contains(output);
+
+  }
+}
+
+class RestCall extends Thread{
+  static Random random = new Random();
+  IncomingMessageEnvelope _envelope;
+  MessageCollector _messageCollector;
+  TaskCallback _callback;
+  RestCall(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCallback callback){
+    this._envelope = envelope;
+    this._callback = callback;
+    this._messageCollector = collector;
+  }
+  @Override
+  public void run(){
+    System.out.println("Running " +  this.getName());
+    try {
+        // Let the thread sleep for a while.
+        Thread.sleep(random.nextInt(150));
+    } catch (InterruptedException e) {
+      System.out.println("Thread " +  this.getName() + " interrupted.");
+    }
+    System.out.println("Thread " +  this.getName() + " exiting.");
+    Integer obj = (Integer) _envelope.getMessage();
+    _messageCollector.send(new OutgoingMessageEnvelope(new SystemStream("test", "Output"), obj * 10));
+    _callback.complete();
+  }
+
+}
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/examples/SampleStreamApplicationTest.java b/samza-test/src/main/java/org/apache/samza/test/framework/examples/SampleStreamApplicationTest.java
new file mode 100644
index 0000000000..050a99d55a
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/examples/SampleStreamApplicationTest.java
@@ -0,0 +1,40 @@
+package org.apache.samza.test.framework.examples;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.system.framework.utils.StreamAssert;
+import org.apache.samza.test.framework.stream.CollectionStream;
+import org.junit.Test;
+
+
+class MyStreamApplication implements StreamApplication {
+  @Override
+  public void init(StreamGraph graph, Config config) {
+    MessageStream pageViews = graph.getInputStream("input");
+
+    pageViews.map(s -> s * 10)
+        .sendTo(graph.getOutputStream("output"));
+  }
+}
+
+public class SampleStreamApplicationTest {
+  @Test
+  public void testStreamApplication() throws Exception{
+    // Create a sample data
+    List input = Arrays.asList(1,2,3,4,5);
+    List output = Arrays.asList(10,20,30,40,50);
+//
+//    TestRunner
+//        .of(new MyStreamApplication())
+//        .addInputStream(CollectionStream.of("test.input",input))
+//        .addOutputStream(CollectionStream.empty("test.output"))
+//        .run();
+
+    StreamAssert.that("test.output").containsInAnyOrder(output);
+
+  }
+}
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/examples/SampleSyncMultiPartitionTest.java b/samza-test/src/main/java/org/apache/samza/test/framework/examples/SampleSyncMultiPartitionTest.java
new file mode 100644
index 0000000000..545d0ee92d
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/examples/SampleSyncMultiPartitionTest.java
@@ -0,0 +1,63 @@
+package org.apache.samza.test.framework.examples;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.framework.utils.StreamAssert;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.StreamTask;
+import org.apache.samza.task.TaskCoordinator;
+import org.apache.samza.test.framework.Mode;
+import org.apache.samza.test.framework.stream.CollectionStream;
+import org.junit.Test;
+import scala.Int;
+
+
+public class SampleSyncMultiPartitionTest {
+
+  @Test
+  public void testSampleSyncMultiPartitionTestWithConcurrency() throws Exception {
+    // Create a sample data
+    Map> input = new HashMap>();
+    Map> expected = new HashMap>();
+
+    for (int i = 0; i < 2; i++) {
+      input.put(i,new ArrayList());
+      expected.put(i, new ArrayList());
+      for (int j = 0; j < 4; j++) {
+        input.get(i).add(j);
+        expected.get(i).add(j * 10);
+      }
+    }
+    System.out.println(Arrays.asList(input));
+    System.out.println(Arrays.asList(expected));
+
+    // Create a StreamTask
+    StreamTask task = new StreamTask() {
+      @Override
+      public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator)
+          throws Exception {
+        Integer obj = (Integer) envelope.getMessage();
+        collector.send(new OutgoingMessageEnvelope(new SystemStream("test", "Output"),
+            Integer.valueOf(envelope.getSystemStreamPartition().getPartition().getPartitionId()), envelope.getKey(),
+            obj * 10));
+      }
+    };
+
+    // Run the test framework
+//    TestRunner
+//        .of(task)
+//        .addInputStream(CollectionStream.of("test.Integer", input))
+//        .addOutputStream(CollectionStream.empty("test.Output"))
+//        .run();
+
+    StreamAssert.that("test.Output").contains(expected);
+  }
+
+}
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/examples/SampleSyncTaskTest.java b/samza-test/src/main/java/org/apache/samza/test/framework/examples/SampleSyncTaskTest.java
new file mode 100644
index 0000000000..050b264a44
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/examples/SampleSyncTaskTest.java
@@ -0,0 +1,106 @@
+package org.apache.samza.test.framework.examples;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import org.apache.samza.serializers.StringSerde;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.StreamTask;
+import org.apache.samza.task.TaskCoordinator;
+import org.apache.samza.test.framework.TestRunner;
+import org.apache.samza.test.framework.stream.CollectionStream;
+import org.apache.samza.test.framework.stream.StreamDescriptor;
+import org.hamcrest.collection.IsIterableContainingInOrder;
+import org.junit.Assert;
+import org.junit.Test;
+
+
+public class SampleSyncTaskTest {
+
+  @Test
+  public void testSyncTaskWithConcurrency() throws Exception{
+    // Create a sample data
+    List input = Arrays.asList(1,2,3,4,5);
+    List output = Arrays.asList(10,20,30,40,50);
+
+    // Create a StreamTask and pass it to factory
+    StreamTask task = new StreamTask() {
+      @Override
+      public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) throws Exception {
+        Integer obj = (Integer)envelope.getMessage();
+        collector.send(new OutgoingMessageEnvelope(new SystemStream("test","output"), obj * 10));
+      }
+    };
+
+
+    CollectionStream intStream = CollectionStream.of("test","input", input);
+    CollectionStream out =  CollectionStream.empty("test", "output");
+
+    TestRunner
+        .of(task)
+        .addInputStream(intStream)
+        .addOutputStream(out)
+        .run();
+
+    Assert.assertThat(out.getStreamState(),IsIterableContainingInOrder.contains(output.toArray()));
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+//    StreamDescriptor.Input input = StreamDescriptor.input("input")
+//        .withKeySerde(new StringSerde("UTF-8"))
+//        .withMsgSerde(new StringSerde("UTF-8"));
+//
+//    StreamDescriptor.Output output = StreamDescriptor.output("output")
+//        .withKeySerde(new StringSerde("UTF-8"))
+//        .withMsgSerde(new StringSerde("UTF-8"));
+//
+//    InMemoryTestSystem testingSystem = InMemoryTestSystem.create("samza-test")
+//        .addInput("input" /*replaced by passing StreamDescriptor*/, inputList)
+//        .addOutput("output" /*replaced by passing StreamDescriptor*/);
+//
+//
+//    // JOB Specific Config
+//    configs.put(JobConfig.JOB_NAME(), JOB_NAME);
+//
+//    // Default Single Container configs
+//    configs.putIfAbsent(JobConfig.PROCESSOR_ID(), "1");
+//    configs.putIfAbsent(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, PassthroughCoordinationUtilsFactory.class.getName());
+//    configs.putIfAbsent(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, PassthroughJobCoordinatorFactory.class.getName());
+//    configs.putIfAbsent(TaskConfig.GROUPER_FACTORY(), SingleContainerGrouperFactory.class.getName());
+//
+//
+//    StreamTaskApplication app = StreamApplications.createStreamTaskApp(config, new MyStreamTaskFactory());
+//    app.addInputs(input).addOutputs(output).runAsTest();
+//
+//    Assert.assertThat(testingSystem.getStreamState("input"), IsIterableContainingInOrder.contains(inputList.toArray()));
+
+  }
+}
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/stream/CollectionStream.java b/samza-test/src/main/java/org/apache/samza/test/framework/stream/CollectionStream.java
new file mode 100644
index 0000000000..3a74c5e06a
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/stream/CollectionStream.java
@@ -0,0 +1,102 @@
+package org.apache.samza.test.framework.stream;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.samza.config.TaskConfig;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.test.framework.InMemoryCollectionStreamSystem;
+import org.apache.samza.test.framework.TestRunner;
+import scala.concurrent.TaskRunner;
+
+
+public class CollectionStream{
+
+  private String streamId;
+  private String systemName;
+  private Map> partitions;
+
+  // Configs only specific to the stream
+  private Map streamConfig;
+
+  // Serdes for the Stream
+  private Serde keySerde;
+  private Serde msgSerde;
+
+  // Configs specific to streams
+  private static final String STREAM_TO_SYSTEM = "streams.%s.samza.system";
+  private static final String KEY_SERDE = "streams.%s.samza.key.serde";
+  private static final String MSG_SERDE = "streams.%s.samza.msg.serde";
+
+
+  private CollectionStream(String systemName, String streamId) {
+    this.streamId = streamId;
+    this.streamConfig = new HashMap<>();
+    this.systemName = systemName;
+    // Config Specific to stream
+    streamConfig.put(String.format(STREAM_TO_SYSTEM, this.streamId), systemName);
+  }
+
+  private CollectionStream(String systemName, String streamId, Iterable collection) {
+    this.streamId = streamId;
+    this.streamConfig = new HashMap<>();
+    this.systemName = systemName;
+    partitions = new HashMap<>();
+    partitions.put(0, collection);
+
+    // Config Specific to stream
+    streamConfig.put(String.format(STREAM_TO_SYSTEM, this.streamId), systemName);
+    streamConfig.put(TaskConfig.INPUT_STREAMS(), systemName+"."+streamId);
+
+  }
+
+  public Map> getInitPartitions(){
+    return partitions;
+  }
+
+  public String getSystemName() {
+    return systemName;
+  }
+
+  public void setSystemName(String systemName) {
+    this.systemName = systemName;
+  }
+
+  public String getStreamId() {
+    return streamId;
+  }
+
+  public Map getStreamConfig() {
+    return streamConfig;
+  }
+
+  public CollectionStream withKeySerde(Serde serde){
+    this.keySerde = serde;
+    return this;
+  }
+
+  public CollectionStream withMsgSerde(Serde serde){
+    this.msgSerde = serde;
+    return this;
+  }
+
+  public  List getStreamState(){
+    InMemoryCollectionStreamSystem system = TestRunner.getOrIntializeInMemoryCollectionStreamSystem(systemName);
+    try {
+      return system.getStreamState(streamId);
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+    return null;
+  }
+
+  public static  CollectionStream empty(String systemName, String streamId) {
+    return new CollectionStream<>(systemName, streamId);
+  }
+
+  public static  CollectionStream of(String systemName, String streamId, Iterable collection) {
+    return new CollectionStream<>(systemName, streamId, collection);
+  }
+
+}
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/stream/EventStream.java b/samza-test/src/main/java/org/apache/samza/test/framework/stream/EventStream.java
new file mode 100644
index 0000000000..af23f83333
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/stream/EventStream.java
@@ -0,0 +1,17 @@
+package org.apache.samza.test.framework.stream;
+
+
+public class EventStream {
+
+  public static abstract class Builder {
+    public abstract Builder addElement();
+    public abstract Builder addException();
+    public abstract Builder advanceTimeTo(long time);
+    public abstract EventStream build();
+  }
+
+  public static  Builder builder() {
+    return null;
+  }
+
+}
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/stream/FileStream.java b/samza-test/src/main/java/org/apache/samza/test/framework/stream/FileStream.java
new file mode 100644
index 0000000000..e27a2a5636
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/stream/FileStream.java
@@ -0,0 +1,5 @@
+package org.apache.samza.test.framework.stream;
+
+public class FileStream {
+  public static  FileStream of(String fileUri) {return null;}
+}
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/stream/StreamDescriptor.java b/samza-test/src/main/java/org/apache/samza/test/framework/stream/StreamDescriptor.java
new file mode 100644
index 0000000000..94d7860d72
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/stream/StreamDescriptor.java
@@ -0,0 +1,44 @@
+package org.apache.samza.test.framework.stream;
+
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.serializers.StringSerde;
+
+
+public class StreamDescriptor {
+  String streamId;
+
+  public String getStreamId() {
+    return streamId;
+  }
+
+  public void setStreamId(String streamId) {
+    this.streamId = streamId;
+  }
+
+  public static Input input(String streamId) {
+    return new Input();
+  }
+
+  public static Output output(String streamId) {
+    return new Output();
+  }
+
+  public static class Input{
+
+    public Input withKeySerde(Serde stringSerde) {
+      return this;
+    }
+    public Input withMsgSerde(Serde stringSerde) {
+      return this;
+    }
+  }
+  public static class Output{
+    public Output withKeySerde(Serde stringSerde) {
+      return this;
+    }
+
+    public Output withMsgSerde(Serde stringSerde) {
+      return this;
+    }
+  }
+}
diff --git a/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
index 29c509dd00..6a30f6341f 100644
--- a/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
+++ b/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
@@ -35,6 +35,13 @@
 import org.apache.samza.runtime.LocalApplicationRunner;
 import org.apache.samza.standalone.PassthroughCoordinationUtilsFactory;
 import org.apache.samza.standalone.PassthroughJobCoordinatorFactory;
+import org.apache.samza.system.IncomingMessageEnvelope;
+import org.apache.samza.system.OutgoingMessageEnvelope;
+import org.apache.samza.system.StreamSpec;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.task.MessageCollector;
+import org.apache.samza.task.StreamTask;
+import org.apache.samza.task.TaskCoordinator;
 import org.apache.samza.test.controlmessages.TestData.PageView;
 import org.apache.samza.test.controlmessages.TestData.PageViewJsonSerdeFactory;
 import org.apache.samza.test.harness.AbstractIntegrationTestHarness;
@@ -57,11 +64,11 @@ public class EndOfStreamIntegrationTest extends AbstractIntegrationTestHarness {
   public void testPipeline() throws  Exception {
     Random random = new Random();
     int count = 10;
-    PageView[] pageviews = new PageView[count];
+    ArrayList pageviews = new ArrayList<>();
     for (int i = 0; i < count; i++) {
       String pagekey = PAGEKEYS[random.nextInt(PAGEKEYS.length - 1)];
       int memberId = random.nextInt(10);
-      pageviews[i] = new PageView(pagekey, memberId);
+      pageviews.add(new PageView(pagekey, memberId));
     }
 
     int partitionCount = 4;
diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/BroadcastAssertApp.java b/samza-test/src/test/java/org/apache/samza/test/operator/BroadcastAssertApp.java
index 9c89aba006..3b034f30ca 100644
--- a/samza-test/src/test/java/org/apache/samza/test/operator/BroadcastAssertApp.java
+++ b/samza-test/src/test/java/org/apache/samza/test/operator/BroadcastAssertApp.java
@@ -25,7 +25,7 @@
 import org.apache.samza.operators.StreamGraph;
 import org.apache.samza.serializers.JsonSerdeV2;
 import org.apache.samza.test.operator.data.PageView;
-import org.apache.samza.test.util.StreamAssert;
+import org.apache.samza.system.framework.utils.StreamAssert;
 
 import java.util.Arrays;
 
diff --git a/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java b/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java
index 04497bd931..97422fd688 100644
--- a/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java
+++ b/samza-test/src/test/java/org/apache/samza/test/operator/StreamApplicationIntegrationTestHarness.java
@@ -32,7 +32,7 @@
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.runtime.ApplicationRunner;
 import org.apache.samza.test.harness.AbstractIntegrationTestHarness;
-import org.apache.samza.test.util.StreamAssert;
+import org.apache.samza.system.framework.utils.StreamAssert;
 import scala.Option;
 import scala.Option$;
 
diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java b/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java
index 8f7eb5d3fd..23fa9e6cae 100644
--- a/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java
+++ b/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java
@@ -74,7 +74,7 @@ public void testSendTo() throws  Exception {
     Profile[] profiles = TestTableData.generateProfiles(count);
 
     int partitionCount = 4;
-    Map configs = getBaseJobConfig();
+    Map configs = getBaseJobConfig(bootstrapUrl(), zkConnect());
 
     configs.put("streams.Profile.samza.system", "test");
     configs.put("streams.Profile.source", Base64Serializer.serialize(profiles));
@@ -112,7 +112,7 @@ public void testStreamTableJoin() throws Exception {
     Profile[] profiles = TestTableData.generateProfiles(count);
 
     int partitionCount = 4;
-    Map configs = getBaseJobConfig();
+    Map configs = getBaseJobConfig(bootstrapUrl(), zkConnect());
 
     configs.put("streams.PageView.samza.system", "test");
     configs.put("streams.PageView.source", Base64Serializer.serialize(pageViews));
@@ -170,7 +170,7 @@ public void testDualStreamTableJoin() throws Exception {
     Profile[] profiles = TestTableData.generateProfiles(count);
 
     int partitionCount = 4;
-    Map configs = getBaseJobConfig();
+    Map configs = getBaseJobConfig(bootstrapUrl(), zkConnect());
 
     configs.put("streams.Profile1.samza.system", "test");
     configs.put("streams.Profile1.source", Base64Serializer.serialize(profiles));
@@ -239,7 +239,7 @@ public void testDualStreamTableJoin() throws Exception {
     assertTrue(joinedPageViews2.get(0) instanceof EnrichedPageView);
   }
 
-  private Map getBaseJobConfig() {
+  static Map getBaseJobConfig(String bootstrapUrl, String zkConnect) {
     Map configs = new HashMap<>();
     configs.put("systems.test.samza.factory", ArraySystemFactory.class.getName());
 
@@ -251,8 +251,8 @@ private Map getBaseJobConfig() {
 
     // For intermediate streams
     configs.put("systems.kafka.samza.factory", "org.apache.samza.system.kafka.KafkaSystemFactory");
-    configs.put("systems.kafka.producer.bootstrap.servers", bootstrapUrl());
-    configs.put("systems.kafka.consumer.zookeeper.connect", zkConnect());
+    configs.put("systems.kafka.producer.bootstrap.servers", bootstrapUrl);
+    configs.put("systems.kafka.consumer.zookeeper.connect", zkConnect);
     configs.put("systems.kafka.samza.key.serde", "int");
     configs.put("systems.kafka.samza.msg.serde", "json");
     configs.put("systems.kafka.default.stream.replication.factor", "1");
@@ -281,7 +281,7 @@ public KV apply(Profile profile) {
     }
   }
 
-  private class PageViewToProfileJoinFunction implements StreamTableJoinFunction
+  static class PageViewToProfileJoinFunction implements StreamTableJoinFunction
       , KV, EnrichedPageView> {
     private int count;
     @Override
diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java b/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java
new file mode 100644
index 0000000000..a260c3f249
--- /dev/null
+++ b/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java
@@ -0,0 +1,180 @@
+/*
+ * 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.test.table;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import org.apache.samza.SamzaException;
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.container.SamzaContainerContext;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.MetricsRegistry;
+import org.apache.samza.metrics.Timer;
+import org.apache.samza.operators.KV;
+import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.serializers.NoOpSerde;
+import org.apache.samza.table.Table;
+import org.apache.samza.table.remote.TableReadFunction;
+import org.apache.samza.table.remote.TableWriteFunction;
+import org.apache.samza.table.remote.RemoteReadableTable;
+import org.apache.samza.table.remote.RemoteTableDescriptor;
+import org.apache.samza.table.remote.RemoteReadWriteTable;
+import org.apache.samza.task.TaskContext;
+import org.apache.samza.test.harness.AbstractIntegrationTestHarness;
+import org.apache.samza.test.util.Base64Serializer;
+import org.apache.samza.util.RateLimiter;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+
+
+public class TestRemoteTable extends AbstractIntegrationTestHarness {
+  private TableReadFunction getInMemoryReader(TestTableData.Profile[] profiles) {
+    final Map profileMap = Arrays.stream(profiles)
+        .collect(Collectors.toMap(p -> p.getMemberId(), Function.identity()));
+    TableReadFunction reader =
+        (TableReadFunction) key -> profileMap.getOrDefault(key, null);
+    return reader;
+  }
+
+  static List writtenRecords = new LinkedList<>();
+
+  static class InMemoryWriteFunction implements TableWriteFunction {
+    private transient List records;
+
+    // Verify serializable functionality
+    private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+      in.defaultReadObject();
+
+      // Write to the global list for verification
+      records = writtenRecords;
+    }
+
+    @Override
+    public void put(Integer key, TestTableData.EnrichedPageView record) {
+      records.add(record);
+    }
+
+    @Override
+    public void delete(Integer key) {
+      records.remove(key);
+    }
+
+    @Override
+    public void deleteAll(Collection keys) {
+      records.removeAll(keys);
+    }
+  }
+
+  @Test
+  public void testStreamTableJoinRemoteTable() throws Exception {
+    List received = new LinkedList<>();
+    final InMemoryWriteFunction writer = new InMemoryWriteFunction();
+
+    int count = 10;
+    TestTableData.PageView[] pageViews = TestTableData.generatePageViews(count);
+    TestTableData.Profile[] profiles = TestTableData.generateProfiles(count);
+
+    int partitionCount = 4;
+    Map configs = TestLocalTable.getBaseJobConfig(bootstrapUrl(), zkConnect());
+
+    configs.put("streams.PageView.samza.system", "test");
+    configs.put("streams.PageView.source", Base64Serializer.serialize(pageViews));
+    configs.put("streams.PageView.partitionCount", String.valueOf(partitionCount));
+
+    final RateLimiter readRateLimiter = mock(RateLimiter.class);
+    final RateLimiter writeRateLimiter = mock(RateLimiter.class);
+    final LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(configs));
+    final StreamApplication app = (streamGraph, cfg) -> {
+      RemoteTableDescriptor inputTableDesc = new RemoteTableDescriptor<>("profile-table-1");
+      inputTableDesc
+          .withReadFunction(getInMemoryReader(profiles))
+          .withRateLimiter(readRateLimiter, null, null);
+
+      RemoteTableDescriptor outputTableDesc = new RemoteTableDescriptor<>("enriched-page-view-table-1");
+      outputTableDesc
+          .withReadFunction(key -> null) // dummy reader
+          .withWriteFunction(writer)
+          .withRateLimiter(writeRateLimiter, null, null);
+
+      Table> inputTable = streamGraph.getTable(inputTableDesc);
+      Table> outputTable = streamGraph.getTable(outputTableDesc);
+
+      streamGraph.getInputStream("PageView", new NoOpSerde())
+          .map(pv -> {
+              received.add(pv);
+              return new KV(pv.getMemberId(), pv);
+            })
+          .join(inputTable, new TestLocalTable.PageViewToProfileJoinFunction())
+          .map(m -> new KV(m.getMemberId(), m))
+          .sendTo(outputTable);
+    };
+
+    runner.run(app);
+    runner.waitForFinish();
+
+    int numExpected = count * partitionCount;
+    Assert.assertEquals(numExpected, received.size());
+    Assert.assertEquals(numExpected, writtenRecords.size());
+    Assert.assertTrue(writtenRecords.get(0) instanceof TestTableData.EnrichedPageView);
+  }
+
+  private TaskContext createMockTaskContext() {
+    MetricsRegistry metricsRegistry = mock(MetricsRegistry.class);
+    doReturn(new Counter("")).when(metricsRegistry).newCounter(anyString(), anyString());
+    doReturn(new Timer("")).when(metricsRegistry).newTimer(anyString(), anyString());
+    TaskContext context = mock(TaskContext.class);
+    doReturn(metricsRegistry).when(context).getMetricsRegistry();
+    return context;
+  }
+
+  @Test(expected = SamzaException.class)
+  public void testCatchReaderException() {
+    TableReadFunction reader = mock(TableReadFunction.class);
+    doThrow(new RuntimeException("Expected test exception")).when(reader).get(anyString());
+    RemoteReadableTable table = new RemoteReadableTable<>("table1", reader, null, null);
+    table.init(mock(SamzaContainerContext.class), createMockTaskContext());
+    table.get("abc");
+  }
+
+  @Test(expected = SamzaException.class)
+  public void testCatchWriterException() {
+    TableReadFunction reader = mock(TableReadFunction.class);
+    TableWriteFunction writer = mock(TableWriteFunction.class);
+    doThrow(new RuntimeException("Expected test exception")).when(writer).put(anyString(), any());
+    RemoteReadWriteTable table = new RemoteReadWriteTable<>("table1", reader, writer, null, null, null);
+    table.init(mock(SamzaContainerContext.class), createMockTaskContext());
+    table.put("abc", "efg");
+  }
+}
diff --git a/samza-test/src/test/java/org/apache/samza/test/timer/TestTimerApp.java b/samza-test/src/test/java/org/apache/samza/test/timer/TestTimerApp.java
new file mode 100644
index 0000000000..13f3c67d46
--- /dev/null
+++ b/samza-test/src/test/java/org/apache/samza/test/timer/TestTimerApp.java
@@ -0,0 +1,86 @@
+/*
+ * 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.test.timer;
+
+import org.apache.samza.application.StreamApplication;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.StreamGraph;
+import org.apache.samza.operators.TimerRegistry;
+import org.apache.samza.operators.functions.FlatMapFunction;
+import org.apache.samza.operators.functions.TimerFunction;
+import org.apache.samza.serializers.JsonSerdeV2;
+import org.apache.samza.test.operator.data.PageView;
+import org.apache.samza.system.framework.utils.StreamAssert;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+public class TestTimerApp implements StreamApplication {
+  public static final String PAGE_VIEWS = "page-views";
+
+  @Override
+  public void init(StreamGraph graph, Config config) {
+    final JsonSerdeV2 serde = new JsonSerdeV2<>(PageView.class);
+    final MessageStream pageViews = graph.getInputStream(PAGE_VIEWS, serde);
+    final MessageStream output = pageViews.flatMap(new FlatmapTimerFn());
+
+    StreamAssert.that("Output from timer function should container all complete messages", output, serde)
+        .containsInAnyOrder(
+            Arrays.asList(
+                new PageView("v1-complete", "p1", "u1"),
+                new PageView("v2-complete", "p2", "u1"),
+                new PageView("v3-complete", "p1", "u2"),
+                new PageView("v4-complete", "p3", "u2")
+            ));
+  }
+
+  private static class FlatmapTimerFn implements FlatMapFunction, TimerFunction {
+
+    private List pageViews = new ArrayList<>();
+    private TimerRegistry timerRegistry;
+
+    @Override
+    public void registerTimer(TimerRegistry timerRegistry) {
+      this.timerRegistry = timerRegistry;
+    }
+
+    @Override
+    public Collection apply(PageView message) {
+      final PageView pv = new PageView(message.getViewId() + "-complete", message.getPageId(), message.getUserId());
+      pageViews.add(pv);
+
+      if (pageViews.size() == 2) {
+        //got all messages for this task
+        final long time = System.currentTimeMillis() + 100;
+        timerRegistry.register("CompleteTimer", time);
+      }
+      return Collections.emptyList();
+    }
+
+    @Override
+    public Collection onTimer(String key, long time) {
+      return pageViews;
+    }
+  }
+}
diff --git a/samza-test/src/test/java/org/apache/samza/test/timer/TimerTest.java b/samza-test/src/test/java/org/apache/samza/test/timer/TimerTest.java
new file mode 100644
index 0000000000..11b3aebbb8
--- /dev/null
+++ b/samza-test/src/test/java/org/apache/samza/test/timer/TimerTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.test.timer;
+
+import org.apache.samza.test.operator.StreamApplicationIntegrationTestHarness;
+import org.junit.Before;
+import org.junit.Test;
+
+
+import static org.apache.samza.test.timer.TestTimerApp.PAGE_VIEWS;
+
+public class TimerTest extends StreamApplicationIntegrationTestHarness {
+
+  @Before
+  public void setup() {
+    // create topics
+    createTopic(PAGE_VIEWS, 2);
+
+    // create events for the following user activity.
+    // userId: (viewId, pageId, (adIds))
+    // u1: (v1, p1, (a1)), (v2, p2, (a3))
+    // u2: (v3, p1, (a1)), (v4, p3, (a5))
+    produceMessage(PAGE_VIEWS, 0, "p1", "{\"viewId\":\"v1\",\"pageId\":\"p1\",\"userId\":\"u1\"}");
+    produceMessage(PAGE_VIEWS, 1, "p2", "{\"viewId\":\"v2\",\"pageId\":\"p2\",\"userId\":\"u1\"}");
+    produceMessage(PAGE_VIEWS, 0, "p1", "{\"viewId\":\"v3\",\"pageId\":\"p1\",\"userId\":\"u2\"}");
+    produceMessage(PAGE_VIEWS, 1, "p3", "{\"viewId\":\"v4\",\"pageId\":\"p3\",\"userId\":\"u2\"}");
+
+  }
+
+  @Test
+  public void testJob() {
+    runApplication(new TestTimerApp(), "TimerTest", null);
+  }
+}
diff --git a/settings.gradle b/settings.gradle
index 06141ac14d..40f9d92e51 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -19,13 +19,14 @@
 
 include \
   'samza-api',
+  'samza-aws',
+  'samza-azure',
   'samza-elasticsearch',
+  'samza-inmemory',
   'samza-log4j',
   'samza-rest',
   'samza-shell',
-  'samza-azure',
   'samza-sql',
-  'samza-aws',
   'samza-tools'
 
 def scalaModules = [
diff --git a/sonar-project.properties b/sonar-project.properties
index 706d99cd51..72dd74380c 100644
--- a/sonar-project.properties
+++ b/sonar-project.properties
@@ -5,15 +5,15 @@
 # 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.
-# 
+#
 
 sonar.projectKey=org.apache.samza
 sonar.projectName=Apache Samza
@@ -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,samza-log4j,samza-rest,samza-shell,samza-test,samza-yarn
+sonar.modules=samza-api,samza-autoscaling,samza-azure,samza-core,samza-elasticsearch,samza-inmemory,samza-hdfs,samza-kafka,samza-kv-inmemory,samza-kv-rocksdb,samza-kv,samza-log4j,samza-rest,samza-shell,samza-test,samza-yarn