From 7946b2f35c0e3811ba7d706fc9c761d73ece47bb Mon Sep 17 00:00:00 2001 From: Damian Guy Date: Mon, 16 Jan 2017 11:40:47 -0800 Subject: [PATCH 001/177] MINOR: Remove unused constructor param from ProcessorStateManager Remove applicationId parameter as it is no longer used. Author: Damian Guy Reviewers: Guozhang Wang Closes #2385 from dguy/minor-remove-unused-param --- .../processor/internals/AbstractTask.java | 2 +- .../internals/ProcessorStateManager.java | 3 +-- .../internals/ProcessorStateManagerTest.java | 16 ++++++++-------- 3 files changed, 10 insertions(+), 11 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java index 0730c68ca0c9f..55418d5b4e030 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java @@ -70,7 +70,7 @@ protected AbstractTask(TaskId id, // create the processor state manager try { - this.stateMgr = new ProcessorStateManager(applicationId, id, partitions, restoreConsumer, isStandby, stateDirectory, topology.storeToChangelogTopic()); + this.stateMgr = new ProcessorStateManager(id, partitions, restoreConsumer, isStandby, stateDirectory, topology.storeToChangelogTopic()); } catch (IOException e) { throw new ProcessorStateException(String.format("task [%s] Error while creating the state manager", id), e); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index a21c3e8dc4f92..ad16c77c2a8ba 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -74,8 +74,7 @@ public class ProcessorStateManager implements StateManager { * (this might be recoverable by retrying) * @throws IOException if any severe error happens while creating or locking the state directory */ - public ProcessorStateManager(final String applicationId, - final TaskId taskId, + public ProcessorStateManager(final TaskId taskId, final Collection sources, final Consumer restoreConsumer, final boolean isStandby, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java index de547230ae3f9..602601a17e731 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java @@ -212,7 +212,7 @@ public void cleanup() { public void testNoTopic() throws IOException { MockStateStoreSupplier.MockStateStore mockStateStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); - ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, new TaskId(0, 1), noPartitions, new MockRestoreConsumer(), false, stateDirectory, new HashMap() { + ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 1), noPartitions, new MockRestoreConsumer(), false, stateDirectory, new HashMap() { { put(nonPersistentStoreName, nonPersistentStoreName); } @@ -244,7 +244,7 @@ public void testRegisterPersistentStore() throws IOException { MockStateStoreSupplier.MockStateStore persistentStore = new MockStateStoreSupplier.MockStateStore("persistentStore", true); // persistent store - ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, taskId, noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { + ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { { put(persistentStoreName, persistentStoreTopicName); put(nonPersistentStoreName, nonPersistentStoreName); @@ -298,7 +298,7 @@ public void testRegisterNonPersistentStore() throws IOException { MockStateStoreSupplier.MockStateStore nonPersistentStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); // non persistent store - ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, new TaskId(0, 2), noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { + ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 2), noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { { put(persistentStoreName, persistentStoreTopicName); put(nonPersistentStoreName, nonPersistentStoreTopicName); @@ -381,7 +381,7 @@ public void testChangeLogOffsets() throws IOException { // if there is an source partition, inherit the partition id Set sourcePartitions = Utils.mkSet(new TopicPartition(storeTopicName3, 1)); - ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, taskId, sourcePartitions, restoreConsumer, true, stateDirectory, storeToChangelogTopic); // standby + ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, sourcePartitions, restoreConsumer, true, stateDirectory, storeToChangelogTopic); // standby try { restoreConsumer.reset(); @@ -415,7 +415,7 @@ public void testGetStore() throws IOException { MockStateStoreSupplier.MockStateStore mockStateStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); - ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, new TaskId(0, 1), noPartitions, restoreConsumer, false, stateDirectory, Collections.emptyMap()); + ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 1), noPartitions, restoreConsumer, false, stateDirectory, Collections.emptyMap()); try { stateMgr.register(mockStateStore, true, mockStateStore.stateRestoreCallback); @@ -453,7 +453,7 @@ public void testFlushAndClose() throws IOException { MockStateStoreSupplier.MockStateStore persistentStore = new MockStateStoreSupplier.MockStateStore(persistentStoreName, true); MockStateStoreSupplier.MockStateStore nonPersistentStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); - ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, taskId, noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { + ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { { put(persistentStoreName, persistentStoreTopicName); put(nonPersistentStoreName, nonPersistentStoreTopicName); @@ -491,7 +491,7 @@ public void testFlushAndClose() throws IOException { @Test public void shouldRegisterStoreWithoutLoggingEnabledAndNotBackedByATopic() throws Exception { MockStateStoreSupplier.MockStateStore mockStateStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); - ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, new TaskId(0, 1), noPartitions, new MockRestoreConsumer(), false, stateDirectory, Collections.emptyMap()); + ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 1), noPartitions, new MockRestoreConsumer(), false, stateDirectory, Collections.emptyMap()); stateMgr.register(mockStateStore, false, mockStateStore.stateRestoreCallback); assertNotNull(stateMgr.getStore(nonPersistentStoreName)); } @@ -512,7 +512,7 @@ public void shouldNotWriteCheckpointsIfAckeOffsetsIsNull() throws Exception { final MockStateStoreSupplier.MockStateStore persistentStore = new MockStateStoreSupplier.MockStateStore(persistentStoreName, true); - final ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, taskId, noPartitions, restoreConsumer, false, stateDirectory, Collections.emptyMap()); + final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, noPartitions, restoreConsumer, false, stateDirectory, Collections.emptyMap()); restoreConsumer.reset(); stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); From 621dff22e79dc64b9a8748186dd985774044f91a Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Tue, 17 Jan 2017 11:16:29 +0000 Subject: [PATCH 002/177] KAFKA-4363; Documentation for sasl.jaas.config property Author: Rajini Sivaram Reviewers: Ismael Juma Closes #2316 from rajinisivaram/KAFKA-4363 --- docs/security.html | 147 ++++++++++++++++++++++++++++++--------------- 1 file changed, 99 insertions(+), 48 deletions(-) diff --git a/docs/security.html b/docs/security.html index 350f0cf620700..8cb867eb04cee 100644 --- a/docs/security.html +++ b/docs/security.html @@ -263,23 +263,65 @@

7.3 Authentication using SASL

SASL configuration for Kafka clients

SASL authentication is only supported for the new Java Kafka producer and - consumer, the older API is not supported. To configure SASL authentication - on the clients: + consumer, the older API is not supported. JAAS configuration for clients may + be specified as a static JAAS config file or using the client configuration property + sasl.jaas.config. + To configure SASL authentication on the clients:
  1. Select a SASL mechanism for authentication.
  2. -
  3. Add a JAAS config file for the selected mechanism as described in the examples - for setting up GSSAPI (Kerberos) - or PLAIN. KafkaClient is the - section name in the JAAS file used by Kafka clients.
  4. -
  5. Pass the JAAS config file location as JVM parameter to each client JVM. For example: -
        -Djava.security.auth.login.config=/etc/kafka/kafka_client_jaas.conf
  6. Configure the following properties in producer.properties or consumer.properties:
        security.protocol=SASL_PLAINTEXT (or SASL_SSL)
    -        sasl.mechanism=GSSAPI (or PLAIN)
  7. + sasl.mechanism=GSSAPI (or PLAIN)
  8. Follow the steps in GSSAPI (Kerberos) or PLAIN to configure SASL for the selected mechanism.
  9. +
  10. Configure JAAS using client configuration property + or static JAAS config file as described below.
  11. +
+
    +
  1. JAAS configuration using client configuration property
    +

    Clients may specify JAAS configuration as a producer or consumer property without + creating a physical configuration file. This mode also enables different producers + and consumers within the same JVM to use different credentials by specifying + different properties for each client. If both static JAAS configuration system property + java.security.auth.login.config and client property sasl.jaas.config + are specified, the client property will be used.

    + + To configure SASL authentication on the clients using configuration property: +
      +
    1. Configure the property sasl.jaas.config in producer.properties or + consumer.properties to be the JAAS login module section of the selected mechanism. + For example, PLAIN + credentials may be configured as: +
          sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule required username="alice" password="alice-secret";
    2. + See GSSAPI (Kerberos) or PLAIN + for full example configurations. +
    +
  2. +
  3. JAAS configuration using static config file
    + To configure SASL authentication on the clients using static JAAS config file: +
      +
    1. Add a JAAS config file with a client login section named KafkaClient. Configure + a login module in KafkaClient for the selected mechanism as described in the examples + for setting up GSSAPI (Kerberos) + or PLAIN. + For example, GSSAPI + credentials may be configured as: +
      +    KafkaClient {
      +        com.sun.security.auth.module.Krb5LoginModule required
      +        useKeyTab=true
      +        storeKey=true
      +        keyTab="/etc/security/keytabs/kafka_client.keytab"
      +        principal="kafka-client-1@EXAMPLE.COM";
      +    };
      + See GSSAPI (Kerberos) or PLAIN + for example configurations of each mechanism.
    2. +
    3. Pass the JAAS config file location as JVM parameter to each client JVM. For example: +
          -Djava.security.auth.login.config=/etc/kafka/kafka_client_jaas.conf
    4. +
    +
  • Authentication using SASL/Kerberos

    @@ -320,7 +362,7 @@

    7.3 Authentication using SASL KafkaServer section in the JAAS file tells the broker which principal to use and the location of the keytab where this principal is stored. It allows the broker to login using the keytab specified in this section. See notes for more details on Zookeeper SASL configuration. -
  • Pass the JAAS and optionally the krb5 file locations as JVM parameters to each Kafka broker (see here for more details): +
  • Pass the JAAS and optionally the krb5 file locations as JVM parameters to each Kafka broker (see here for more details):
        -Djava.security.krb5.conf=/etc/kafka/krb5.conf
             -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
  • @@ -331,7 +373,7 @@

    7.3 Authentication using SASL -

  • We must also configure the service name in server.properties, which should match the principal name of the kafka brokers. In the above example, principal is "kafka/kafka1.hostname.com@EXAMPLE.com", so: + We must also configure the service name in server.properties, which should match the principal name of the kafka brokers. In the above example, principal is "kafka/kafka1.hostname.com@EXAMPLE.com", so:
        sasl.kerberos.service.name=kafka
    @@ -339,37 +381,43 @@

    7.3 Authentication using SASL
  • - Clients (producers, consumers, connect workers, etc) will authenticate to the cluster with their own principal (usually with the same name as the user running the client), so obtain or create these principals as needed. Then create a JAAS file for each principal. - The KafkaClient section describes how the clients like producer and consumer can connect to the Kafka Broker. The following is an example configuration for a client using a keytab (recommended for long-running processes): + Clients (producers, consumers, connect workers, etc) will authenticate to the cluster with their + own principal (usually with the same name as the user running the client), so obtain or create + these principals as needed. Then configure the JAAS configuration property for each client. + Different clients within a JVM may run as different users by specifiying different principals. + The property sasl.jaas.config in producer.properties or consumer.properties describes + how clients like producer and consumer can connect to the Kafka Broker. The following is an example + configuration for a client using a keytab (recommended for long-running processes):
    -        KafkaClient {
    -            com.sun.security.auth.module.Krb5LoginModule required
    -            useKeyTab=true
    -            storeKey=true
    -            keyTab="/etc/security/keytabs/kafka_client.keytab"
    -            principal="kafka-client-1@EXAMPLE.COM";
    -        };
    - - For command-line utilities like kafka-console-consumer or kafka-console-producer, kinit can be used along with "useTicketCache=true" as in: + sasl.jaas.config=com.sun.security.auth.module.Krb5LoginModule required \ + useKeyTab=true \ + storeKey=true \ + keyTab="/etc/security/keytabs/kafka_client.keytab" \ + principal="kafka-client-1@EXAMPLE.COM"; + + For command-line utilities like kafka-console-consumer or kafka-console-producer, kinit can be used + along with "useTicketCache=true" as in:
    -        KafkaClient {
    -            com.sun.security.auth.module.Krb5LoginModule required
    -            useTicketCache=true;
    -        };
    + sasl.jaas.config=com.sun.security.auth.module.Krb5LoginModule required \ + useTicketCache=true; + + JAAS configuration for clients may alternatively be specified as a JVM parameter similar to brokers + as described here. Clients use the login section named + KafkaClient. This option allows only one user for all client connections from a JVM.
  • +
  • Make sure the keytabs configured in the JAAS configuration are readable by the operating system user who is starting kafka client.
  • -
  • Pass the JAAS and optionally krb5 file locations as JVM parameters to each client JVM (see here for more details): -
        -Djava.security.krb5.conf=/etc/kafka/krb5.conf
    -        -Djava.security.auth.login.config=/etc/kafka/kafka_client_jaas.conf
  • -
  • Make sure the keytabs configured in the kafka_client_jaas.conf are readable by the operating system user who is starting kafka client.
  • -
  • Configure the following properties in producer.properties or consumer.properties: -
        security.protocol=SASL_PLAINTEXT (or SASL_SSL)
    -        sasl.mechanism=GSSAPI
    -        sasl.kerberos.service.name=kafka
  • +
  • Optionally pass the krb5 file locations as JVM parameters to each client JVM (see here for more details): +
        -Djava.security.krb5.conf=/etc/kafka/krb5.conf
  • +
  • Configure the following properties in producer.properties or consumer.properties: +
    +    security.protocol=SASL_PLAINTEXT (or SASL_SSL)
    +    sasl.mechanism=GSSAPI
    +    sasl.kerberos.service.name=kafka
  • - +
  • Authentication using SASL/PLAIN

    SASL/PLAIN is a simple username/password authentication mechanism that is typically used with TLS for encryption to implement secure authentication. Kafka supports a default implementation for SASL/PLAIN which can be extended for production use as described here.

    @@ -404,22 +452,25 @@

    7.3 Authentication using SASL

    Configuring Kafka Clients
    To configure SASL authentication on the clients:
      -
    1. The KafkaClient section describes how the clients like producer and consumer can connect to the Kafka Broker. - The following is an example configuration for a client for the PLAIN mechanism: +
    2. Configure the JAAS configuration property for each client in producer.properties or consumer.properties. + The login module describes how the clients like producer and consumer can connect to the Kafka Broker. + The following is an example configuration for a client for the PLAIN mechanism:
      -        KafkaClient {
      -            org.apache.kafka.common.security.plain.PlainLoginModule required
      -            username="alice"
      -            password="alice-secret";
      -        };
      - The properties username and password in the KafkaClient section are used by clients to configure + sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule required \ + username="alice" \ + password="alice-secret"; +

      The options username and password are used by clients to configure the user for client connections. In this example, clients connect to the broker as user alice. -

    3. -
    4. Pass the JAAS config file location as JVM parameter to each client JVM: -
          -Djava.security.auth.login.config=/etc/kafka/kafka_client_jaas.conf
    5. + Different clients within a JVM may connect as different users by specifying different user names + and passwords in sasl.jaas.config.

      + +

      JAAS configuration for clients may alternatively be specified as a JVM parameter similar to brokers + as described here. Clients use the login section named + KafkaClient. This option allows only one user for all client connections from a JVM.

    6. Configure the following properties in producer.properties or consumer.properties: -
          security.protocol=SASL_SSL
      -        sasl.mechanism=PLAIN
    7. +
      +    security.protocol=SASL_SSL
      +    sasl.mechanism=PLAIN
  • Use of SASL/PLAIN in production
    From e3f4cdd0e249f78a7f4e8f064533bcd15eb11cbf Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Tue, 17 Jan 2017 12:55:07 +0000 Subject: [PATCH 003/177] KAFKA-4590; SASL/SCRAM system tests Runs sanity test and one replication test using SASL/SCRAM. Author: Rajini Sivaram Reviewers: Ewen Cheslack-Postava , Ismael Juma Closes #2355 from rajinisivaram/KAFKA-4590 --- .../sanity_checks/test_console_consumer.py | 2 +- tests/kafkatest/services/kafka/kafka.py | 6 +++++ .../services/security/security_config.py | 25 +++++++++++++++++++ .../services/security/templates/jaas.conf | 9 +++++++ .../kafkatest/tests/core/replication_test.py | 6 ++++- 5 files changed, 46 insertions(+), 2 deletions(-) diff --git a/tests/kafkatest/sanity_checks/test_console_consumer.py b/tests/kafkatest/sanity_checks/test_console_consumer.py index 38db057e5654a..066d6d42c1477 100644 --- a/tests/kafkatest/sanity_checks/test_console_consumer.py +++ b/tests/kafkatest/sanity_checks/test_console_consumer.py @@ -47,7 +47,7 @@ def setUp(self): @parametrize(security_protocol='PLAINTEXT', new_consumer=False) @matrix(security_protocol=['PLAINTEXT', 'SSL']) @cluster(num_nodes=4) - @parametrize(security_protocol='SASL_SSL', sasl_mechanism='PLAIN') + @matrix(security_protocol=['SASL_SSL'], sasl_mechanism=['PLAIN', 'SCRAM-SHA-256', 'SCRAM-SHA-512']) @matrix(security_protocol=['SASL_PLAINTEXT', 'SASL_SSL']) def test_lifecycle(self, security_protocol, new_consumer=True, sasl_mechanism='GSSAPI'): """Check that console consumer starts/stops properly, and that we are capturing log output.""" diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 716c2d2645402..8ef0f356f43cc 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -208,6 +208,7 @@ def start_node(self, node): node.account.create_file(self.LOG4J_CONFIG, self.render('log4j.properties', log_dir=KafkaService.OPERATIONAL_LOG_DIR)) self.security_config.setup_node(node) + self.security_config.setup_credentials(node, self.path, self.zk.connect_setting(), broker=True) cmd = self.start_cmd(node) self.logger.debug("Attempting to start KafkaService on %s with command: %s" % (str(node.account), cmd)) @@ -215,6 +216,11 @@ def start_node(self, node): node.account.ssh(cmd) monitor.wait_until("Kafka Server.*started", timeout_sec=30, backoff_sec=.25, err_msg="Kafka server didn't finish startup") + # Credentials for inter-broker communication are created before starting Kafka. + # Client credentials are created after starting Kafka so that both loading of + # existing credentials from ZK and dynamic update of credentials in Kafka are tested. + self.security_config.setup_credentials(node, self.path, self.zk.connect_setting(), broker=False) + self.start_jmx_tool(self.idx(node), node) if len(self.pids(node)) == 0: raise Exception("No process ids recorded on node %s" % str(node)) diff --git a/tests/kafkatest/services/security/security_config.py b/tests/kafkatest/services/security/security_config.py index 9b29217bf723e..864e0a3008484 100644 --- a/tests/kafkatest/services/security/security_config.py +++ b/tests/kafkatest/services/security/security_config.py @@ -94,6 +94,12 @@ class SecurityConfig(TemplateRenderer): SASL_SSL = 'SASL_SSL' SASL_MECHANISM_GSSAPI = 'GSSAPI' SASL_MECHANISM_PLAIN = 'PLAIN' + SASL_MECHANISM_SCRAM_SHA_256 = 'SCRAM-SHA-256' + SASL_MECHANISM_SCRAM_SHA_512 = 'SCRAM-SHA-512' + SCRAM_CLIENT_USER = "kafka-client" + SCRAM_CLIENT_PASSWORD = "client-secret" + SCRAM_BROKER_USER = "kafka-broker" + SCRAM_BROKER_PASSWORD = "broker-secret" CONFIG_DIR = "/mnt/security" KEYSTORE_PATH = "/mnt/security/test.keystore.jks" TRUSTSTORE_PATH = "/mnt/security/test.truststore.jks" @@ -167,6 +173,7 @@ def setup_sasl(self, node): else: is_ibm_jdk = False jaas_conf = self.render(jaas_conf_file, node=node, is_ibm_jdk=is_ibm_jdk, + SecurityConfig=SecurityConfig, client_sasl_mechanism=self.client_sasl_mechanism, enabled_sasl_mechanisms=self.enabled_sasl_mechanisms) node.account.create_file(SecurityConfig.JAAS_CONF_PATH, jaas_conf) @@ -181,6 +188,21 @@ def setup_node(self, node): if self.has_sasl: self.setup_sasl(node) + def setup_credentials(self, node, path, zk_connect, broker): + if broker: + self.maybe_create_scram_credentials(node, zk_connect, path, self.interbroker_sasl_mechanism, + SecurityConfig.SCRAM_BROKER_USER, SecurityConfig.SCRAM_BROKER_PASSWORD) + else: + self.maybe_create_scram_credentials(node, zk_connect, path, self.client_sasl_mechanism, + SecurityConfig.SCRAM_CLIENT_USER, SecurityConfig.SCRAM_CLIENT_PASSWORD) + + def maybe_create_scram_credentials(self, node, zk_connect, path, mechanism, user_name, password): + if self.has_sasl and self.is_sasl_scram(mechanism): + cmd = "%s --zookeeper %s --entity-name %s --entity-type users --alter --add-config %s=[password=%s]" % \ + (path.script("kafka-configs.sh", node), zk_connect, + user_name, mechanism, password) + node.account.ssh(cmd) + def clean_node(self, node): if self.security_protocol != SecurityConfig.PLAINTEXT: node.account.ssh("rm -rf %s" % SecurityConfig.CONFIG_DIR, allow_fail=False) @@ -203,6 +225,9 @@ def is_ssl(self, security_protocol): def is_sasl(self, security_protocol): return security_protocol == SecurityConfig.SASL_PLAINTEXT or security_protocol == SecurityConfig.SASL_SSL + def is_sasl_scram(self, sasl_mechanism): + return sasl_mechanism == SecurityConfig.SASL_MECHANISM_SCRAM_SHA_256 or sasl_mechanism == SecurityConfig.SASL_MECHANISM_SCRAM_SHA_512 + @property def security_protocol(self): return self.properties['security.protocol'] diff --git a/tests/kafkatest/services/security/templates/jaas.conf b/tests/kafkatest/services/security/templates/jaas.conf index fbfa8af49c2db..3667f87b661a0 100644 --- a/tests/kafkatest/services/security/templates/jaas.conf +++ b/tests/kafkatest/services/security/templates/jaas.conf @@ -31,6 +31,10 @@ KafkaClient { org.apache.kafka.common.security.plain.PlainLoginModule required username="client" password="client-secret"; +{% elif client_sasl_mechanism == "SCRAM-SHA-256" or client_sasl_mechanism == "SCRAM-SHA-512" %} + org.apache.kafka.common.security.scram.ScramLoginModule required + username="{{ SecurityConfig.SCRAM_CLIENT_USER }}" + password="{{ SecurityConfig.SCRAM_CLIENT_PASSWORD }}"; {% endif %} }; @@ -58,6 +62,11 @@ KafkaServer { user_client="client-secret" user_kafka="kafka-secret"; {% endif %} +{% if client_sasl_mechanism == "SCRAM-SHA-256" or client_sasl_mechanism == "SCRAM-SHA-512" %} + org.apache.kafka.common.security.scram.ScramLoginModule required + username="{{ SecurityConfig.SCRAM_BROKER_USER }}" + password="{{ SecurityConfig.SCRAM_BROKER_PASSWORD }}"; +{% endif %} }; {% if zk_sasl %} diff --git a/tests/kafkatest/tests/core/replication_test.py b/tests/kafkatest/tests/core/replication_test.py index a95e9e5c289b9..3e17d5636eca3 100644 --- a/tests/kafkatest/tests/core/replication_test.py +++ b/tests/kafkatest/tests/core/replication_test.py @@ -16,6 +16,7 @@ from ducktape.utils.util import wait_until from ducktape.mark import matrix +from ducktape.mark import parametrize from ducktape.mark.resource import cluster from kafkatest.services.zookeeper import ZookeeperService @@ -122,13 +123,16 @@ def min_cluster_size(self): @cluster(num_nodes=7) @matrix(failure_mode=["clean_shutdown", "hard_shutdown", "clean_bounce", "hard_bounce"], broker_type=["leader"], - security_protocol=["PLAINTEXT", "SSL", "SASL_PLAINTEXT", "SASL_SSL"]) + security_protocol=["PLAINTEXT", "SASL_SSL"]) @matrix(failure_mode=["clean_shutdown", "hard_shutdown", "clean_bounce", "hard_bounce"], broker_type=["controller"], security_protocol=["PLAINTEXT", "SASL_SSL"]) @matrix(failure_mode=["hard_bounce"], broker_type=["leader"], security_protocol=["SASL_SSL"], client_sasl_mechanism=["PLAIN"], interbroker_sasl_mechanism=["PLAIN", "GSSAPI"]) + @parametrize(failure_mode="hard_bounce", + broker_type="leader", + security_protocol="SASL_SSL", client_sasl_mechanism="SCRAM-SHA-256", interbroker_sasl_mechanism="SCRAM-SHA-512") def test_replication_with_broker_failure(self, failure_mode, security_protocol, broker_type, client_sasl_mechanism="GSSAPI", interbroker_sasl_mechanism="GSSAPI"): """Replication tests. These tests verify that replication provides simple durability guarantees by checking that data acked by From 2b19ad9d8c47fb0f78a6e90d2f5711df6110bf1f Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Tue, 17 Jan 2017 18:42:55 +0000 Subject: [PATCH 004/177] KAFKA-4580; Use sasl.jaas.config for some system tests Switched console_consumer, verifiable_consumer and verifiable_producer to use new sasl.jaas_config property instead of static JAAS configuration file when used with SASL_PLAINTEXT. Author: Rajini Sivaram Reviewers: Ewen Cheslack-Postava , Ismael Juma Closes #2323 from rajinisivaram/KAFKA-4580 (cherry picked from commit 3f6c4f63c9c17424cf717ca76c74554bcf3b2e9a) Signed-off-by: Ismael Juma --- tests/kafkatest/services/console_consumer.py | 4 +-- .../services/security/security_config.py | 28 +++++++++++++++---- .../services/security/templates/jaas.conf | 4 +++ .../kafkatest/services/verifiable_consumer.py | 5 ++-- .../kafkatest/services/verifiable_producer.py | 9 +++--- 5 files changed, 35 insertions(+), 15 deletions(-) diff --git a/tests/kafkatest/services/console_consumer.py b/tests/kafkatest/services/console_consumer.py index 17ddb6be26aa6..cdc46cdca985d 100644 --- a/tests/kafkatest/services/console_consumer.py +++ b/tests/kafkatest/services/console_consumer.py @@ -150,7 +150,8 @@ def prop_file(self, node): # Add security properties to the config. If security protocol is not specified, # use the default in the template properties. - self.security_config = self.kafka.security_config.client_config(prop_file) + self.security_config = self.kafka.security_config.client_config(prop_file, node) + self.security_config.setup_node(node) prop_file += str(self.security_config) return prop_file @@ -231,7 +232,6 @@ def _worker(self, idx, node): prop_file = self.prop_file(node) self.logger.info(prop_file) node.account.create_file(ConsoleConsumer.CONFIG_FILE, prop_file) - self.security_config.setup_node(node) # Create and upload log properties log_config = self.render('tools_log4j.properties', log_file=ConsoleConsumer.LOG_FILE) diff --git a/tests/kafkatest/services/security/security_config.py b/tests/kafkatest/services/security/security_config.py index 864e0a3008484..846d9b10f3a6a 100644 --- a/tests/kafkatest/services/security/security_config.py +++ b/tests/kafkatest/services/security/security_config.py @@ -112,7 +112,7 @@ class SecurityConfig(TemplateRenderer): def __init__(self, context, security_protocol=None, interbroker_security_protocol=None, client_sasl_mechanism=SASL_MECHANISM_GSSAPI, interbroker_sasl_mechanism=SASL_MECHANISM_GSSAPI, - zk_sasl=False, template_props=""): + zk_sasl=False, template_props="", static_jaas_conf=True): """ Initialize the security properties for the node and copy keystore and truststore to the remote node if the transport protocol @@ -143,6 +143,7 @@ def __init__(self, context, security_protocol=None, interbroker_security_protoco self.has_sasl = self.is_sasl(security_protocol) or self.is_sasl(interbroker_security_protocol) or zk_sasl self.has_ssl = self.is_ssl(security_protocol) or self.is_ssl(interbroker_security_protocol) self.zk_sasl = zk_sasl + self.static_jaas_conf = static_jaas_conf self.properties = { 'security.protocol' : security_protocol, 'ssl.keystore.location' : SecurityConfig.KEYSTORE_PATH, @@ -156,8 +157,14 @@ def __init__(self, context, security_protocol=None, interbroker_security_protoco 'sasl.kerberos.service.name' : 'kafka' } - def client_config(self, template_props=""): - return SecurityConfig(self.context, self.security_protocol, client_sasl_mechanism=self.client_sasl_mechanism, template_props=template_props) + def client_config(self, template_props="", node=None): + # If node is not specified, use static jaas config which will be created later. + # Otherwise use static JAAS configuration files with SASL_SSL and sasl.jaas.config + # property with SASL_PLAINTEXT so that both code paths are tested by existing tests. + # Note that this is an artibtrary choice and it is possible to run all tests with + # either static or dynamic jaas config files if required. + static_jaas_conf = node is None or (self.has_sasl and self.has_ssl) + return SecurityConfig(self.context, self.security_protocol, client_sasl_mechanism=self.client_sasl_mechanism, template_props=template_props, static_jaas_conf=static_jaas_conf) def setup_ssl(self, node): node.account.ssh("mkdir -p %s" % SecurityConfig.CONFIG_DIR, allow_fail=False) @@ -175,8 +182,12 @@ def setup_sasl(self, node): jaas_conf = self.render(jaas_conf_file, node=node, is_ibm_jdk=is_ibm_jdk, SecurityConfig=SecurityConfig, client_sasl_mechanism=self.client_sasl_mechanism, - enabled_sasl_mechanisms=self.enabled_sasl_mechanisms) - node.account.create_file(SecurityConfig.JAAS_CONF_PATH, jaas_conf) + enabled_sasl_mechanisms=self.enabled_sasl_mechanisms, + static_jaas_conf=self.static_jaas_conf) + if self.static_jaas_conf: + node.account.create_file(SecurityConfig.JAAS_CONF_PATH, jaas_conf) + else: + self.properties['sasl.jaas.config'] = jaas_conf.replace("\n", " \\\n") if self.has_sasl_kerberos: node.account.copy_to(MiniKdc.LOCAL_KEYTAB_FILE, SecurityConfig.KEYTAB_PATH) node.account.copy_to(MiniKdc.LOCAL_KRB5CONF_FILE, SecurityConfig.KRB5CONF_PATH) @@ -251,7 +262,10 @@ def has_sasl_kerberos(self): @property def kafka_opts(self): if self.has_sasl: - return "\"-Djava.security.auth.login.config=%s -Djava.security.krb5.conf=%s\"" % (SecurityConfig.JAAS_CONF_PATH, SecurityConfig.KRB5CONF_PATH) + if self.static_jaas_conf: + return "\"-Djava.security.auth.login.config=%s -Djava.security.krb5.conf=%s\"" % (SecurityConfig.JAAS_CONF_PATH, SecurityConfig.KRB5CONF_PATH) + else: + return "\"-Djava.security.krb5.conf=%s\"" % SecurityConfig.KRB5CONF_PATH else: return "" @@ -265,6 +279,8 @@ def props(self, prefix=''): """ if self.security_protocol == SecurityConfig.PLAINTEXT: return "" + if self.has_sasl and not self.static_jaas_conf and 'sasl.jaas.config' not in self.properties: + raise Exception("JAAS configuration property has not yet been initialized") config_lines = (prefix + key + "=" + value for key, value in self.properties.iteritems()) # Extra blank lines ensure this can be appended/prepended safely return "\n".join(itertools.chain([""], config_lines, [""])) diff --git a/tests/kafkatest/services/security/templates/jaas.conf b/tests/kafkatest/services/security/templates/jaas.conf index 3667f87b661a0..e2511451e3272 100644 --- a/tests/kafkatest/services/security/templates/jaas.conf +++ b/tests/kafkatest/services/security/templates/jaas.conf @@ -12,7 +12,9 @@ */ +{% if static_jaas_conf %} KafkaClient { +{% endif %} {% if client_sasl_mechanism == "GSSAPI" %} {% if is_ibm_jdk %} com.ibm.security.auth.module.Krb5LoginModule required debug=false @@ -37,6 +39,7 @@ KafkaClient { password="{{ SecurityConfig.SCRAM_CLIENT_PASSWORD }}"; {% endif %} +{% if static_jaas_conf %} }; KafkaServer { @@ -102,3 +105,4 @@ Server { {% endif %} }; {% endif %} +{% endif %} diff --git a/tests/kafkatest/services/verifiable_consumer.py b/tests/kafkatest/services/verifiable_consumer.py index c593e2a79bd42..090bcda8784fb 100644 --- a/tests/kafkatest/services/verifiable_consumer.py +++ b/tests/kafkatest/services/verifiable_consumer.py @@ -148,8 +148,6 @@ def __init__(self, context, num_nodes, kafka, topic, group_id, self.enable_autocommit = enable_autocommit self.assignment_strategy = assignment_strategy self.prop_file = "" - self.security_config = kafka.security_config.client_config(self.prop_file) - self.prop_file += str(self.security_config) self.stop_timeout_sec = stop_timeout_sec self.event_handlers = {} @@ -171,6 +169,9 @@ def _worker(self, idx, node): node.account.create_file(VerifiableConsumer.LOG4J_CONFIG, log_config) # Create and upload config file + self.security_config = self.kafka.security_config.client_config(self.prop_file, node) + self.security_config.setup_node(node) + self.prop_file += str(self.security_config) self.logger.info("verifiable_consumer.properties:") self.logger.info(self.prop_file) node.account.create_file(VerifiableConsumer.CONFIG_FILE, self.prop_file) diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py index 205143e411839..d873e1f82b525 100644 --- a/tests/kafkatest/services/verifiable_producer.py +++ b/tests/kafkatest/services/verifiable_producer.py @@ -83,10 +83,6 @@ def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput self.acks = acks self.stop_timeout_sec = stop_timeout_sec - @property - def security_config(self): - return self.kafka.security_config.client_config() - def prop_file(self, node): idx = self.idx(node) prop_file = str(self.security_config) @@ -104,6 +100,10 @@ def _worker(self, idx, node): log_config = self.render('tools_log4j.properties', log_file=VerifiableProducer.LOG_FILE) node.account.create_file(VerifiableProducer.LOG4J_CONFIG, log_config) + # Configure security + self.security_config = self.kafka.security_config.client_config(node=node) + self.security_config.setup_node(node) + # Create and upload config file producer_prop_file = self.prop_file(node) if self.acks is not None: @@ -112,7 +112,6 @@ def _worker(self, idx, node): self.logger.info("verifiable_producer.properties:") self.logger.info(producer_prop_file) node.account.create_file(VerifiableProducer.CONFIG_FILE, producer_prop_file) - self.security_config.setup_node(node) cmd = self.start_cmd(node, idx) self.logger.debug("VerifiableProducer %d command: %s" % (idx, cmd)) From 60d759a227087079e6fd270c68fd9e38441cb34a Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 17 Jan 2017 10:42:05 -0800 Subject: [PATCH 005/177] MINOR: Some cleanups and additional testing for KIP-88 Author: Jason Gustafson Reviewers: Vahid Hashemian , Ismael Juma Closes #2383 from hachikuji/minor-cleanup-kip-88 --- .../common/requests/OffsetFetchRequest.java | 24 +++++- .../common/requests/OffsetFetchResponse.java | 83 ++++++------------ .../clients/consumer/KafkaConsumerTest.java | 2 +- .../internals/ConsumerCoordinatorTest.java | 4 +- .../common/requests/RequestResponseTest.java | 23 +++-- .../scala/kafka/api/OffsetFetchRequest.scala | 1 - .../kafka/coordinator/GroupCoordinator.scala | 4 +- .../coordinator/GroupMetadataManager.scala | 31 ++++--- .../main/scala/kafka/server/KafkaApis.scala | 86 ++++++++----------- .../kafka/api/AuthorizerIntegrationTest.scala | 37 +++++++- .../GroupCoordinatorResponseTest.scala | 58 ++++++++++++- 11 files changed, 211 insertions(+), 142 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java index 0ff49bed09c2d..553fd9695c3b6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java @@ -23,6 +23,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -78,7 +79,7 @@ public String toString() { private final List partitions; public static OffsetFetchRequest forAllPartitions(String groupId) { - return new OffsetFetchRequest.Builder(groupId, (List) null).setVersion((short) 2).build(); + return new OffsetFetchRequest.Builder(groupId, null).setVersion((short) 2).build(); } // v0, v1, and v2 have the same fields. @@ -131,20 +132,35 @@ public OffsetFetchRequest(Struct struct, short versionId) { groupId = struct.getString(GROUP_ID_KEY_NAME); } - @Override - public AbstractResponse getErrorResponse(Throwable e) { + public OffsetFetchResponse getErrorResponse(Errors error) { short versionId = version(); + + Map responsePartitions = new HashMap<>(); + if (versionId < 2) { + for (TopicPartition partition : this.partitions) { + responsePartitions.put(partition, new OffsetFetchResponse.PartitionData( + OffsetFetchResponse.INVALID_OFFSET, + OffsetFetchResponse.NO_METADATA, + error)); + } + } + switch (versionId) { case 0: case 1: case 2: - return new OffsetFetchResponse(Errors.forException(e), partitions, versionId); + return new OffsetFetchResponse(error, responsePartitions, versionId); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_FETCH.id))); } } + @Override + public OffsetFetchResponse getErrorResponse(Throwable e) { + return getErrorResponse(Errors.forException(e)); + } + public String groupId() { return groupId; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java index 0095f38ec10f5..9c1415538c54f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -46,6 +46,8 @@ public class OffsetFetchResponse extends AbstractResponse { public static final long INVALID_OFFSET = -1L; public static final String NO_METADATA = ""; + public static final PartitionData UNKNOWN_PARTITION = new PartitionData(INVALID_OFFSET, NO_METADATA, + Errors.UNKNOWN_TOPIC_OR_PARTITION); /** * Possible error codes: @@ -59,7 +61,7 @@ public class OffsetFetchResponse extends AbstractResponse { * - GROUP_AUTHORIZATION_FAILED (30) */ - public static final List PARTITION_ERRORS = Arrays.asList( + private static final List PARTITION_ERRORS = Arrays.asList( Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.TOPIC_AUTHORIZATION_FAILED); @@ -82,14 +84,30 @@ public boolean hasError() { } } - private List getTopicArray(Map responseData) { - Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + /** + * Constructor for the latest version. + * @param error Potential coordinator or group level error code + * @param responseData Fetched offset information grouped by topic-partition + */ + public OffsetFetchResponse(Errors error, Map responseData) { + this(error, responseData, CURRENT_VERSION); + } - List topicArray = new ArrayList(); + /** + * Unified constructor for all versions. + * @param error Potential coordinator or group level error code (for api version 2 and later) + * @param responseData Fetched offset information grouped by topic-partition + * @param version The request API version + */ + public OffsetFetchResponse(Errors error, Map responseData, int version) { + super(new Struct(ProtoUtils.responseSchema(ApiKeys.OFFSET_FETCH.id, version))); + + Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + List topicArray = new ArrayList<>(); for (Map.Entry> entries : topicsData.entrySet()) { Struct topicData = this.struct.instance(RESPONSES_KEY_NAME); topicData.set(TOPIC_KEY_NAME, entries.getKey()); - List partitionArray = new ArrayList(); + List partitionArray = new ArrayList<>(); for (Map.Entry partitionEntry : entries.getValue().entrySet()) { PartitionData fetchPartitionData = partitionEntry.getValue(); Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); @@ -103,66 +121,17 @@ private List getTopicArray(Map responseDa topicArray.add(topicData); } - return topicArray; - } - - /** - * Unified constructor - * @param responseData Fetched offset information grouped by topic-partition - * @param topLevelErrorCode Potential coordinator or group level error code (for api version 2 and later) - * @param version The request API version - */ - public OffsetFetchResponse(Errors topLevelError, Map responseData, int version) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.OFFSET_FETCH.id, version))); - - this.struct.set(RESPONSES_KEY_NAME, getTopicArray(responseData).toArray()); + this.struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); this.responseData = responseData; - this.error = topLevelError; + this.error = error; if (version > 1) this.struct.set(ERROR_CODE_KEY_NAME, this.error.code()); } - /** - * Unified constructor (used only if there are errors in the response) - * @param partitions partitions to be included in the response - * @param topLevelErrorCode The error code to be reported in the response - * @param version The request API version - */ - public OffsetFetchResponse(Errors topLevelError, List partitions, int version) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.OFFSET_FETCH.id, version))); - - assert topLevelError != Errors.NONE; - this.responseData = new HashMap<>(); - this.error = topLevelError; - if (version < 2) { - for (TopicPartition partition : partitions) { - this.responseData.put(partition, new OffsetFetchResponse.PartitionData( - OffsetFetchResponse.INVALID_OFFSET, - OffsetFetchResponse.NO_METADATA, - topLevelError)); - } - } else - this.struct.set(ERROR_CODE_KEY_NAME, this.error.code()); - - this.struct.set(RESPONSES_KEY_NAME, getTopicArray(this.responseData).toArray()); - } - - public OffsetFetchResponse(Map responseData) { - this(Errors.NONE, responseData, CURRENT_VERSION); - } - - /** - * Constructor for version 2 and above when there is a coordinator or group level error - * @param topLevelErrorCode Coordinator or group level error code - */ - public OffsetFetchResponse(Errors topLevelError) { - this(topLevelError, new ArrayList(), CURRENT_VERSION); - } - public OffsetFetchResponse(Struct struct) { super(struct); Errors topLevelError = Errors.NONE; - this.responseData = new HashMap(); + this.responseData = new HashMap<>(); for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { Struct topicResponse = (Struct) topicResponseObj; String topic = topicResponse.getString(TOPIC_KEY_NAME); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 2eeed551abdb0..d4913df75a9c1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -1370,7 +1370,7 @@ private OffsetFetchResponse offsetResponse(Map offsets, Er for (Map.Entry entry : offsets.entrySet()) { partitionData.put(entry.getKey(), new OffsetFetchResponse.PartitionData(entry.getValue(), "", error)); } - return new OffsetFetchResponse(partitionData); + return new OffsetFetchResponse(Errors.NONE, partitionData); } private ListOffsetResponse listOffsetsResponse(Map offsets, short error) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index 9a8c0b93de6d9..ee6afe19de5ac 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -1482,12 +1482,12 @@ private OffsetCommitResponse offsetCommitResponse(Map res } private OffsetFetchResponse offsetFetchResponse(Errors topLevelError) { - return new OffsetFetchResponse(topLevelError); + return new OffsetFetchResponse(topLevelError, Collections.emptyMap()); } private OffsetFetchResponse offsetFetchResponse(TopicPartition tp, Errors partitionLevelError, String metadata, long offset) { OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, metadata, partitionLevelError); - return new OffsetFetchResponse(Collections.singletonMap(tp, data)); + return new OffsetFetchResponse(Errors.NONE, Collections.singletonMap(tp, data)); } private OffsetCommitCallback callback(final AtomicBoolean success) { diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 0d3a1a8a38053..a5ed806673975 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -43,6 +43,7 @@ import java.util.Map; import java.util.Set; +import static java.util.Collections.singletonList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -90,8 +91,13 @@ public void testSerialization() throws Exception { checkSerialization(createOffsetCommitResponse(), null); checkSerialization(OffsetFetchRequest.forAllPartitions("group1")); checkSerialization(OffsetFetchRequest.forAllPartitions("group1").getErrorResponse(new NotCoordinatorForGroupException()), 2); - checkSerialization(createOffsetFetchRequest()); - checkSerialization(createOffsetFetchRequest().getErrorResponse(new UnknownServerException()), null); + checkSerialization(createOffsetFetchRequest(0)); + checkSerialization(createOffsetFetchRequest(1)); + checkSerialization(createOffsetFetchRequest(2)); + checkSerialization(OffsetFetchRequest.forAllPartitions("group1")); + checkSerialization(createOffsetFetchRequest(0).getErrorResponse(new UnknownServerException()), 0); + checkSerialization(createOffsetFetchRequest(1).getErrorResponse(new UnknownServerException()), 1); + checkSerialization(createOffsetFetchRequest(2).getErrorResponse(new UnknownServerException()), 2); checkSerialization(createOffsetFetchResponse(), null); checkSerialization(createProduceRequest()); checkSerialization(createProduceRequest().getErrorResponse(new UnknownServerException()), null); @@ -337,7 +343,7 @@ private ListGroupsResponse createListGroupsResponse() { } private DescribeGroupsRequest createDescribeGroupRequest() { - return new DescribeGroupsRequest.Builder(Collections.singletonList("test-group")).build(); + return new DescribeGroupsRequest.Builder(singletonList("test-group")).build(); } private DescribeGroupsResponse createDescribeGroupResponse() { @@ -428,16 +434,17 @@ private OffsetCommitResponse createOffsetCommitResponse() { return new OffsetCommitResponse(responseData); } - private OffsetFetchRequest createOffsetFetchRequest() { - return new OffsetFetchRequest.Builder("group1", - Arrays.asList(new TopicPartition("test11", 1))).build(); + private OffsetFetchRequest createOffsetFetchRequest(int version) { + return new OffsetFetchRequest.Builder("group1", singletonList(new TopicPartition("test11", 1))) + .setVersion((short) version) + .build(); } private OffsetFetchResponse createOffsetFetchResponse() { Map responseData = new HashMap<>(); responseData.put(new TopicPartition("test", 0), new OffsetFetchResponse.PartitionData(100L, "", Errors.NONE)); responseData.put(new TopicPartition("test", 1), new OffsetFetchResponse.PartitionData(100L, null, Errors.NONE)); - return new OffsetFetchResponse(responseData); + return new OffsetFetchResponse(Errors.NONE, responseData); } private ProduceRequest createProduceRequest() { @@ -544,7 +551,7 @@ private SaslHandshakeRequest createSaslHandshakeRequest() { } private SaslHandshakeResponse createSaslHandshakeResponse() { - return new SaslHandshakeResponse(Errors.NONE.code(), Collections.singletonList("GSSAPI")); + return new SaslHandshakeResponse(Errors.NONE.code(), singletonList("GSSAPI")); } private ApiVersionsRequest createApiVersionRequest() { diff --git a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala index 2908901cf5aea..dac4cc5d75d4e 100644 --- a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala @@ -25,7 +25,6 @@ import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response import kafka.utils.Logging import org.apache.kafka.common.protocol.{ApiKeys, Errors} -import org.apache.kafka.common.requests.OffsetFetchResponse.PARTITION_ERRORS object OffsetFetchRequest extends Logging { val CurrentVersion: Short = 2 diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index 4cbfad6ade6f6..7abbc6e1ac1dd 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -422,7 +422,7 @@ class GroupCoordinator(val brokerId: Int, } } - def doCommitOffsets(group: GroupMetadata, + private def doCommitOffsets(group: GroupMetadata, memberId: String, generationId: Int, offsetMetadata: immutable.Map[TopicPartition, OffsetAndMetadata], @@ -455,7 +455,7 @@ class GroupCoordinator(val brokerId: Int, } def handleFetchOffsets(groupId: String, - partitions: Option[Seq[TopicPartition]]): (Errors, Map[TopicPartition, OffsetFetchResponse.PartitionData]) = { + partitions: Option[Seq[TopicPartition]] = None): (Errors, Map[TopicPartition, OffsetFetchResponse.PartitionData]) = { if (!isActive.get) (Errors.GROUP_COORDINATOR_NOT_AVAILABLE, Map()) else if (!isCoordinatorForGroup(groupId)) { diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index 74b46adc28f06..2d6889c936ff5 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -97,7 +97,7 @@ class GroupMetadataManager(val brokerId: Int, unit = TimeUnit.MILLISECONDS) } - def currentGroups(): Iterable[GroupMetadata] = groupMetadataCache.values + def currentGroups: Iterable[GroupMetadata] = groupMetadataCache.values def isPartitionOwned(partition: Int) = inLock(partitionLock) { ownedPartitions.contains(partition) } @@ -342,19 +342,24 @@ class GroupMetadataManager(val brokerId: Int, (topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NONE)) }.toMap } else { - if (topicPartitionsOpt.isEmpty) { - // Return offsets for all partitions owned by this consumer group. (this only applies to consumers that commit offsets to Kafka.) - group.allOffsets.map { case (topicPartition, offsetAndMetadata) => - (topicPartition, new OffsetFetchResponse.PartitionData(offsetAndMetadata.offset, offsetAndMetadata.metadata, Errors.NONE)) - } - } else { - topicPartitionsOpt.getOrElse(Seq.empty[TopicPartition]).map { topicPartition => - group.offset(topicPartition) match { - case None => (topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NONE)) - case Some(offsetAndMetadata) => - (topicPartition, new OffsetFetchResponse.PartitionData(offsetAndMetadata.offset, offsetAndMetadata.metadata, Errors.NONE)) + topicPartitionsOpt match { + case None => + // Return offsets for all partitions owned by this consumer group. (this only applies to consumers + // that commit offsets to Kafka.) + group.allOffsets.map { case (topicPartition, offsetAndMetadata) => + topicPartition -> new OffsetFetchResponse.PartitionData(offsetAndMetadata.offset, offsetAndMetadata.metadata, Errors.NONE) } - }.toMap + + case Some(topicPartitions) => + topicPartitionsOpt.getOrElse(Seq.empty[TopicPartition]).map { topicPartition => + val partitionData = group.offset(topicPartition) match { + case None => + new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NONE) + case Some(offsetAndMetadata) => + new OffsetFetchResponse.PartitionData(offsetAndMetadata.offset, offsetAndMetadata.metadata, Errors.NONE) + } + topicPartition -> partitionData + }.toMap } } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index a6ad7b2ec8799..530dafcd69147 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -38,7 +38,7 @@ import kafka.utils.{Logging, ZKGroupTopicDirs, ZkUtils} import org.apache.kafka.common.errors.{ClusterAuthorizationException, NotLeaderForPartitionException, TopicExistsException, UnknownTopicOrPartitionException, UnsupportedForMessageFormatException} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.protocol.{ApiKeys, Errors, Protocol, SecurityProtocol} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, Protocol} import org.apache.kafka.common.record.{MemoryRecords, Record} import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse @@ -886,38 +886,31 @@ class KafkaApis(val requestChannel: RequestChannel, requestChannel.sendResponse(new RequestChannel.Response(request, responseBody)) } - /* + /** * Handle an offset fetch request */ def handleOffsetFetchRequest(request: RequestChannel.Request) { val header = request.header val offsetFetchRequest = request.body.asInstanceOf[OffsetFetchRequest] + def authorizeTopicDescribe(partition: TopicPartition) = + authorize(request.session, Describe, new Resource(auth.Topic, partition.topic)) + val offsetFetchResponse = // reject the request if not authorized to the group if (!authorize(request.session, Read, new Resource(Group, offsetFetchRequest.groupId))) - new OffsetFetchResponse(Errors.GROUP_AUTHORIZATION_FAILED, offsetFetchRequest.partitions, header.apiVersion) + offsetFetchRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED) else { - val partitions = - if (offsetFetchRequest.isAllPartitions) - List[TopicPartition]() - else - offsetFetchRequest.partitions.asScala.toList - - val (authorizedPartitions, unauthorizedPartitions) = - partitions.partition { partition => authorize(request.session, Describe, new Resource(auth.Topic, partition.topic)) } - - val unknownTopicPartitionResponse = new OffsetFetchResponse.PartitionData( - OffsetFetchResponse.INVALID_OFFSET, OffsetFetchResponse.NO_METADATA, Errors.UNKNOWN_TOPIC_OR_PARTITION) - val unauthorizedStatus = unauthorizedPartitions.map(topicPartition => (topicPartition, unknownTopicPartitionResponse)).toMap - if (header.apiVersion == 0) { + val (authorizedPartitions, unauthorizedPartitions) = offsetFetchRequest.partitions.asScala + .partition(authorizeTopicDescribe) + // version 0 reads offsets from ZK - val responseInfo = authorizedPartitions.map { topicPartition => + val authorizedPartitionData = authorizedPartitions.map { topicPartition => val topicDirs = new ZKGroupTopicDirs(offsetFetchRequest.groupId, topicPartition.topic) try { if (!metadataCache.contains(topicPartition.topic)) - (topicPartition, unknownTopicPartitionResponse) + (topicPartition, OffsetFetchResponse.UNKNOWN_PARTITION) else { val payloadOpt = zkUtils.readDataMaybeNull(s"${topicDirs.consumerOffsetDir}/${topicPartition.partition}")._1 payloadOpt match { @@ -925,7 +918,7 @@ class KafkaApis(val requestChannel: RequestChannel, (topicPartition, new OffsetFetchResponse.PartitionData( payload.toLong, OffsetFetchResponse.NO_METADATA, Errors.NONE)) case None => - (topicPartition, unknownTopicPartitionResponse) + (topicPartition, OffsetFetchResponse.UNKNOWN_PARTITION) } } } catch { @@ -934,43 +927,32 @@ class KafkaApis(val requestChannel: RequestChannel, OffsetFetchResponse.INVALID_OFFSET, OffsetFetchResponse.NO_METADATA, Errors.forException(e))) } }.toMap - new OffsetFetchResponse(Errors.NONE, (responseInfo ++ unauthorizedStatus).asJava, header.apiVersion) - } - else { + + val unauthorizedPartitionData = unauthorizedPartitions.map(_ -> OffsetFetchResponse.UNKNOWN_PARTITION).toMap + new OffsetFetchResponse(Errors.NONE, (authorizedPartitionData ++ unauthorizedPartitionData).asJava, header.apiVersion) + } else { // versions 1 and above read offsets from Kafka - val offsets = coordinator.handleFetchOffsets(offsetFetchRequest.groupId, - if (offsetFetchRequest.isAllPartitions) - None - else + if (offsetFetchRequest.isAllPartitions) { + val (error, allPartitionData) = coordinator.handleFetchOffsets(offsetFetchRequest.groupId) + if (error != Errors.NONE) + offsetFetchRequest.getErrorResponse(error) + else { + // clients are not allowed to see offsets for topics that are not authorized for Describe + val authorizedPartitionData = allPartitionData.filter { case (topicPartition, _) => authorizeTopicDescribe(topicPartition) } + new OffsetFetchResponse(Errors.NONE, authorizedPartitionData.asJava, header.apiVersion) + } + } else { + val (authorizedPartitions, unauthorizedPartitions) = offsetFetchRequest.partitions.asScala + .partition(authorizeTopicDescribe) + val (error, authorizedPartitionData) = coordinator.handleFetchOffsets(offsetFetchRequest.groupId, Some(authorizedPartitions)) - - // Note that we do not need to filter the partitions in the - // metadata cache as the topic partitions will be filtered - // in coordinator's offset manager through the offset cache - if (header.apiVersion == 1) { - val authorizedStatus = - if (offsets._1 != Errors.NONE) { - authorizedPartitions.map { partition => - (partition, new OffsetFetchResponse.PartitionData( - OffsetFetchResponse.INVALID_OFFSET, OffsetFetchResponse.NO_METADATA, offsets._1))}.toMap - } - else - offsets._2.toMap - new OffsetFetchResponse(Errors.NONE, (authorizedStatus ++ unauthorizedStatus).asJava, header.apiVersion) - } - else if (offsets._1 == Errors.NONE) { - if (offsetFetchRequest.isAllPartitions) { - // filter out unauthorized topics in case all group offsets are requested - val authorizedStatus = offsets._2.filter { - case (partition, _) => authorize(request.session, Describe, new Resource(auth.Topic, partition.topic)) - } - new OffsetFetchResponse((authorizedStatus).asJava) + if (error != Errors.NONE) + offsetFetchRequest.getErrorResponse(error) + else { + val unauthorizedPartitionData = unauthorizedPartitions.map(_ -> OffsetFetchResponse.UNKNOWN_PARTITION).toMap + new OffsetFetchResponse(Errors.NONE, (authorizedPartitionData ++ unauthorizedPartitionData).asJava, header.apiVersion) } - else - new OffsetFetchResponse((offsets._2.toMap ++ unauthorizedStatus).asJava) } - else - new OffsetFetchResponse(offsets._1) } } diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 42251fa8950cb..d43d1af2a4d2d 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -40,6 +40,7 @@ import scala.collection.mutable import scala.collection.mutable.Buffer import org.apache.kafka.common.KafkaException import kafka.admin.AdminUtils +import kafka.network.SocketServer import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.record.MemoryRecords @@ -173,7 +174,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { producers.foreach(_.close()) consumers.foreach(_.wakeup()) consumers.foreach(_.close()) - removeAllAcls + removeAllAcls() super.tearDown() } @@ -711,6 +712,34 @@ class AuthorizerIntegrationTest extends BaseRequestTest { this.consumers.head.position(tp) } + @Test + def testFetchAllOffsetsTopicAuthorization() { + val offset = 15L + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Read)), groupResource) + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Read)), topicResource) + this.consumers.head.assign(List(tp).asJava) + this.consumers.head.commitSync(Map(tp -> new OffsetAndMetadata(offset)).asJava) + + removeAllAcls() + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Read)), groupResource) + + // send offset fetch requests directly since the consumer does not expose an API to do so + // note there's only one broker, so no need to lookup the group coordinator + + // without describe permission on the topic, we shouldn't be able to fetch offsets + val offsetFetchRequest = requests.OffsetFetchRequest.forAllPartitions(group) + var offsetFetchResponse = sendOffsetFetchRequest(offsetFetchRequest, anySocketServer) + assertEquals(Errors.NONE, offsetFetchResponse.error) + assertTrue(offsetFetchResponse.responseData.isEmpty) + + // now add describe permission on the topic and verify that the offset can be fetched + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe)), topicResource) + offsetFetchResponse = sendOffsetFetchRequest(offsetFetchRequest, anySocketServer) + assertEquals(Errors.NONE, offsetFetchResponse.error) + assertTrue(offsetFetchResponse.responseData.containsKey(tp)) + assertEquals(offset, offsetFetchResponse.responseData.get(tp).offset) + } + @Test def testOffsetFetchTopicDescribe() { addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Read)), groupResource) @@ -846,4 +875,10 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } } + private def sendOffsetFetchRequest(request: requests.OffsetFetchRequest, + socketServer: SocketServer): requests.OffsetFetchResponse = { + val response = send(request, ApiKeys.OFFSET_FETCH, socketServer) + requests.OffsetFetchResponse.parse(response, request.version) + } + } diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala index 20e512faaaaff..d3de16d049761 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala @@ -25,7 +25,7 @@ import kafka.server.{DelayedOperationPurgatory, KafkaConfig, ReplicaManager} import kafka.utils._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest} +import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest, OffsetFetchResponse} import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.easymock.{Capture, EasyMock, IAnswer} import org.junit.{After, Before, Test} @@ -748,6 +748,62 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, commitOffsetResult(tp)) } + @Test + def testFetchOffsets() { + val tp = new TopicPartition("topic", 0) + val offset = OffsetAndMetadata(0) + + val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID, + OffsetCommitRequest.DEFAULT_GENERATION_ID, immutable.Map(tp -> offset)) + assertEquals(Errors.NONE.code, commitOffsetResult(tp)) + + val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp))) + assertEquals(Errors.NONE, error) + assertEquals(Some(0), partitionData.get(tp).map(_.offset)) + } + + @Test + def testFetchOffsetForUnknownPartition(): Unit = { + val tp = new TopicPartition("topic", 0) + val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp))) + assertEquals(Errors.NONE, error) + assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData.get(tp).map(_.offset)) + } + + @Test + def testFetchOffsetNotCoordinatorForGroup(): Unit = { + val tp = new TopicPartition("topic", 0) + val (error, partitionData) = groupCoordinator.handleFetchOffsets(otherGroupId, Some(Seq(tp))) + assertEquals(Errors.NOT_COORDINATOR_FOR_GROUP, error) + assertTrue(partitionData.isEmpty) + } + + @Test + def testFetchAllOffsets() { + val tp1 = new TopicPartition("topic", 0) + val tp2 = new TopicPartition("topic", 1) + val tp3 = new TopicPartition("other-topic", 0) + val offset1 = OffsetAndMetadata(15) + val offset2 = OffsetAndMetadata(16) + val offset3 = OffsetAndMetadata(17) + + assertEquals((Errors.NONE, Map.empty), groupCoordinator.handleFetchOffsets(groupId)) + + val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID, + OffsetCommitRequest.DEFAULT_GENERATION_ID, immutable.Map(tp1 -> offset1, tp2 -> offset2, tp3 -> offset3)) + assertEquals(Errors.NONE.code, commitOffsetResult(tp1)) + assertEquals(Errors.NONE.code, commitOffsetResult(tp2)) + assertEquals(Errors.NONE.code, commitOffsetResult(tp3)) + + val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId) + assertEquals(Errors.NONE, error) + assertEquals(3, partitionData.size) + assertTrue(partitionData.forall(_._2.error == Errors.NONE)) + assertEquals(Some(offset1.offset), partitionData.get(tp1).map(_.offset)) + assertEquals(Some(offset2.offset), partitionData.get(tp2).map(_.offset)) + assertEquals(Some(offset3.offset), partitionData.get(tp3).map(_.offset)) + } + @Test def testCommitOffsetInAwaitingSync() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID From c9b9acf6a8b5433332c2d0d825c17a4a20cf3fa5 Mon Sep 17 00:00:00 2001 From: Damian Guy Date: Tue, 17 Jan 2017 12:33:11 -0800 Subject: [PATCH 006/177] KAFKA-4588: Wait for topics to be created in QueryableStateIntegrationTest.shouldNotMakeStoreAvailableUntilAllStoresAvailable After debugging this i can see the times that it fails there is a race between when the topic is actually created/ready on the broker and when the assignment happens. When it fails `StreamPartitionAssignor.assign(..)` gets called with a `Cluster` with no topics. Hence the test hangs as no tasks get assigned. To fix this I added a `waitForTopics` method to `EmbeddedKafkaCluster`. This will wait until the topics have been created. Author: Damian Guy Reviewers: Matthias J. Sax, Guozhang Wang Closes #2371 from dguy/integration-test-fix (cherry picked from commit 825f225bc5706b16af8ec44ca47ee1452c11e6f3) Signed-off-by: Guozhang Wang --- checkstyle/import-control.xml | 1 + .../GlobalKTableIntegrationTest.java | 4 +- ...StreamAggregationDedupIntegrationTest.java | 4 +- .../KStreamAggregationIntegrationTest.java | 4 +- .../KStreamKTableJoinIntegrationTest.java | 2 +- .../KStreamRepartitionJoinTest.java | 8 ++-- .../QueryableStateIntegrationTest.java | 14 +++--- .../utils/EmbeddedKafkaCluster.java | 24 +++++++++-- .../utils/IntegrationTestUtils.java | 43 +++++++++++++++++++ .../integration/utils/KafkaEmbedded.java | 3 ++ .../apache/kafka/test/StreamsTestUtils.java | 1 + 11 files changed, 87 insertions(+), 21 deletions(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index b68cf988821ab..04f364c6f8c18 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -156,6 +156,7 @@ + diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java index 85b851d12964f..6ac87ae1fe64a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java @@ -81,7 +81,7 @@ public String apply(final Long value1, final String value2) { private ForeachAction foreachAction; @Before - public void before() { + public void before() throws InterruptedException { testNo++; builder = new KStreamBuilder(); createTopics(); @@ -212,7 +212,7 @@ public boolean conditionMet() { } - private void createTopics() { + private void createTopics() throws InterruptedException { inputStream = "input-stream-" + testNo; inputTable = "input-table-" + testNo; globalOne = "globalOne-" + testNo; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java index 9397e0331f474..f2a767ceb514e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java @@ -72,7 +72,7 @@ public class KStreamAggregationDedupIntegrationTest { @Before - public void before() { + public void before() throws InterruptedException { testNo++; builder = new KStreamBuilder(); createTopics(); @@ -267,7 +267,7 @@ private void produceMessages(long timestamp) } - private void createTopics() { + private void createTopics() throws InterruptedException { streamOneInput = "stream-one-" + testNo; outputTopic = "output-" + testNo; CLUSTER.createTopic(streamOneInput, 3, 1); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java index 0833f3c8d1fe4..beb41ce68c610 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java @@ -89,7 +89,7 @@ public class KStreamAggregationIntegrationTest { private KStream stream; @Before - public void before() { + public void before() throws InterruptedException { testNo++; builder = new KStreamBuilder(); createTopics(); @@ -637,7 +637,7 @@ private void produceMessages(final long timestamp) } - private void createTopics() { + private void createTopics() throws InterruptedException { streamOneInput = "stream-one-" + testNo; outputTopic = "output-" + testNo; userSessionsStream = userSessionsStream + "-" + testNo; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKTableJoinIntegrationTest.java index 3618f1542c271..0a16494397c71 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKTableJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKTableJoinIntegrationTest.java @@ -74,7 +74,7 @@ public class KStreamKTableJoinIntegrationTest { private Properties streamsConfiguration; @Before - public void before() { + public void before() throws InterruptedException { testNo++; userClicksTopic = "user-clicks-" + testNo; userRegionsTopic = "user-regions-" + testNo; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java index e8a042ad7c670..6f3c95a3ab536 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java @@ -93,7 +93,7 @@ public static Object[] data() { } @Before - public void before() { + public void before() throws InterruptedException { testNo++; String applicationId = "kstream-repartition-join-test-" + testNo; builder = new KStreamBuilder(); @@ -146,7 +146,7 @@ public void shouldCorrectlyRepartitionOnJoinOperations() throws Exception { verifyLeftJoin(leftJoin); } - private ExpectedOutputOnTopic mapStreamOneAndJoin() { + private ExpectedOutputOnTopic mapStreamOneAndJoin() throws InterruptedException { String mapOneStreamAndJoinOutput = "map-one-join-output-" + testNo; doJoin(streamOne.map(keyMapper), streamTwo, mapOneStreamAndJoinOutput); return new ExpectedOutputOnTopic(expectedStreamOneTwoJoin, mapOneStreamAndJoinOutput); @@ -350,7 +350,7 @@ private void produceToStreamOne() mockTime); } - private void createTopics() { + private void createTopics() throws InterruptedException { streamOneInput = "stream-one-" + testNo; streamTwoInput = "stream-two-" + testNo; streamFourInput = "stream-four-" + testNo; @@ -395,7 +395,7 @@ private void verifyCorrectOutput(final List expectedMessages, private void doJoin(final KStream lhs, final KStream rhs, - final String outputTopic) { + final String outputTopic) throws InterruptedException { CLUSTER.createTopic(outputTopic); lhs.join(rhs, TOSTRING_JOINER, diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java index b8f91faed0341..911c6a89cfe17 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java @@ -81,6 +81,7 @@ public class QueryableStateIntegrationTest { @ClassRule public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS); + public static final int STREAM_THREE_PARTITIONS = 4; private final MockTime mockTime = CLUSTER.time; private String streamOne = "stream-one"; private String streamTwo = "stream-two"; @@ -91,7 +92,7 @@ public class QueryableStateIntegrationTest { private String outputTopicThree = "output-three"; // sufficiently large window size such that everything falls into 1 window private static final long WINDOW_SIZE = TimeUnit.MILLISECONDS.convert(2, TimeUnit.DAYS); - private static final int NUM_PARTITIONS = 2; + private static final int STREAM_TWO_PARTITIONS = 2; private static final int NUM_REPLICAS = NUM_BROKERS; private Properties streamsConfiguration; private List inputValues; @@ -101,7 +102,7 @@ public class QueryableStateIntegrationTest { private Comparator> stringLongComparator; private static int testNo = 0; - public void createTopics() { + public void createTopics() throws InterruptedException { streamOne = streamOne + "-" + testNo; streamConcurrent = streamConcurrent + "-" + testNo; streamThree = streamThree + "-" + testNo; @@ -111,8 +112,8 @@ public void createTopics() { streamTwo = streamTwo + "-" + testNo; CLUSTER.createTopic(streamOne); CLUSTER.createTopic(streamConcurrent); - CLUSTER.createTopic(streamTwo, NUM_PARTITIONS, NUM_REPLICAS); - CLUSTER.createTopic(streamThree, 4, 1); + CLUSTER.createTopic(streamTwo, STREAM_TWO_PARTITIONS, NUM_REPLICAS); + CLUSTER.createTopic(streamThree, STREAM_THREE_PARTITIONS, 1); CLUSTER.createTopic(outputTopic); CLUSTER.createTopic(outputTopicConcurrent); CLUSTER.createTopic(outputTopicThree); @@ -128,7 +129,7 @@ public static Object[] data() { } @Before - public void before() throws IOException { + public void before() throws IOException, InterruptedException { testNo++; createTopics(); streamsConfiguration = new Properties(); @@ -145,7 +146,6 @@ public void before() throws IOException { streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, cacheSizeBytes); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); - stringComparator = new Comparator>() { @Override @@ -328,7 +328,7 @@ public boolean conditionMet() { @Test public void queryOnRebalance() throws Exception { - final int numThreads = NUM_PARTITIONS; + final int numThreads = STREAM_TWO_PARTITIONS; final StreamRunnable[] streamRunnables = new StreamRunnable[numThreads]; final Thread[] streamThreads = new Thread[numThreads]; final int numIterations = 500000; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java index 519b1f5cb14b6..619b6b5ed87e3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java @@ -18,13 +18,17 @@ package org.apache.kafka.streams.integration.utils; import kafka.server.KafkaConfig$; +import kafka.server.KafkaServer; import kafka.utils.MockTime; import kafka.zk.EmbeddedZookeeper; +import org.apache.kafka.common.TopicPartition; import org.junit.rules.ExternalResource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import java.util.Properties; /** @@ -34,6 +38,7 @@ public class EmbeddedKafkaCluster extends ExternalResource { private static final Logger log = LoggerFactory.getLogger(EmbeddedKafkaCluster.class); private static final int DEFAULT_BROKER_PORT = 0; // 0 results in a random port being selected + public static final int TOPIC_CREATION_TIMEOUT = 30000; private EmbeddedZookeeper zookeeper = null; private final KafkaEmbedded[] brokers; private final Properties brokerConfig; @@ -122,7 +127,7 @@ protected void after() { * * @param topic The name of the topic. */ - public void createTopic(final String topic) { + public void createTopic(final String topic) throws InterruptedException { createTopic(topic, 1, 1, new Properties()); } @@ -133,7 +138,7 @@ public void createTopic(final String topic) { * @param partitions The number of partitions for this topic. * @param replication The replication factor for (the partitions of) this topic. */ - public void createTopic(final String topic, final int partitions, final int replication) { + public void createTopic(final String topic, final int partitions, final int replication) throws InterruptedException { createTopic(topic, partitions, replication, new Properties()); } @@ -148,11 +153,24 @@ public void createTopic(final String topic, final int partitions, final int repl public void createTopic(final String topic, final int partitions, final int replication, - final Properties topicConfig) { + final Properties topicConfig) throws InterruptedException { brokers[0].createTopic(topic, partitions, replication, topicConfig); + final List topicPartitions = new ArrayList<>(); + for (int partition = 0; partition < partitions; partition++) { + topicPartitions.add(new TopicPartition(topic, partition)); + } + IntegrationTestUtils.waitForTopicPartitions(brokers(), topicPartitions, TOPIC_CREATION_TIMEOUT); } public void deleteTopic(final String topic) { brokers[0].deleteTopic(topic); } + + public List brokers() { + final List servers = new ArrayList<>(); + for (final KafkaEmbedded broker : brokers) { + servers.add(broker.kafkaServer()); + } + return servers; + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java index aa358ab3322e6..875c3591b881b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java @@ -17,6 +17,10 @@ package org.apache.kafka.streams.integration.utils; +import kafka.api.PartitionStateInfo; +import kafka.api.Request; +import kafka.server.KafkaServer; +import kafka.server.MetadataCache; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -24,12 +28,14 @@ import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; +import scala.Option; import java.io.File; import java.io.IOException; @@ -255,4 +261,41 @@ public boolean conditionMet() { return accumData; } + public static void waitForTopicPartitions(final List servers, + final List partitions, + final long timeout) throws InterruptedException { + final long end = System.currentTimeMillis() + timeout; + for (final TopicPartition partition : partitions) { + final long remaining = end - System.currentTimeMillis(); + if (remaining <= 0) { + throw new AssertionError("timed out while waiting for partitions to become available. Timeout=" + timeout); + } + waitUntilMetadataIsPropagated(servers, partition.topic(), partition.partition(), remaining); + } + } + + public static void waitUntilMetadataIsPropagated(final List servers, + final String topic, + final int partition, + final long timeout) throws InterruptedException { + TestUtils.waitForCondition(new TestCondition() { + @Override + public boolean conditionMet() { + for (final KafkaServer server : servers) { + final MetadataCache metadataCache = server.apis().metadataCache(); + final Option partitionInfo = + metadataCache.getPartitionInfo(topic, partition); + if (partitionInfo.isEmpty()) { + return false; + } + final PartitionStateInfo partitionStateInfo = partitionInfo.get(); + if (!Request.isValidBrokerId(partitionStateInfo.leaderIsrAndControllerEpoch().leaderAndIsr().leader())) { + return false; + } + } + return true; + } + }, timeout, "metatadata for topic=" + topic + " partition=" + partition + " not propogated to all brokers"); + + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java index 70c50631deedd..9b48272343e4a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java @@ -202,4 +202,7 @@ public void deleteTopic(final String topic) { zkClient.close(); } + public KafkaServer kafkaServer() { + return kafka; + } } diff --git a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java index 555e622237535..73c1b63a2a47b 100644 --- a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java @@ -66,4 +66,5 @@ public static List> toList(final Iterator> } return results; } + } From 6f72a5a53c444278187fa6be58031168bcaffb26 Mon Sep 17 00:00:00 2001 From: Damian Guy Date: Tue, 17 Jan 2017 14:13:46 -0800 Subject: [PATCH 007/177] KAFKA-3452 Follow-up: Refactoring StateStore hierarchies This is a follow up of https://github.com/apache/kafka/pull/2166 - refactoring the store hierarchies as requested Author: Damian Guy Reviewers: Guozhang Wang Closes #2360 from dguy/state-store-refactor (cherry picked from commit 73b7ae0019d387407375f3865e263225c986a6ce) Signed-off-by: Guozhang Wang --- .../kstream/internals/SessionKeySerde.java | 17 ++ .../streams/state/WindowStoreIterator.java | 3 +- ...bstractMergedSortedCacheStoreIterator.java | 166 ++++++++++++++ .../state/internals/CachingKeyValueStore.java | 10 +- .../state/internals/CachingSessionStore.java | 63 ++---- .../state/internals/CachingWindowStore.java | 64 ++---- .../ChangeLoggingKeyValueBytesStore.java | 93 ++++++++ .../internals/ChangeLoggingKeyValueStore.java | 127 +++++++++++ .../ChangeLoggingSegmentedBytesStore.java | 28 +-- .../CompositeReadOnlyWindowStore.java | 5 + .../DelegatingPeekingKeyValueIterator.java | 10 +- ...ergedSortedCacheKeyValueStoreIterator.java | 130 ++--------- ...MergedSortedCacheSessionStoreIterator.java | 71 ++++++ .../MergedSortedCacheWindowStoreIterator.java | 58 +++++ ...MergedSortedCachedWindowStoreIterator.java | 107 --------- .../state/internals/MeteredKeyValueStore.java | 27 +-- .../internals/MeteredSegmentedBytesStore.java | 27 +-- .../state/internals/MeteredWindowStore.java | 180 --------------- .../RocksDBKeyValueStoreSupplier.java | 54 +++-- .../RocksDBSessionStoreSupplier.java | 54 +++-- .../streams/state/internals/RocksDBStore.java | 28 +-- .../state/internals/RocksDBWindowStore.java | 25 ++- .../internals/RocksDBWindowStoreSupplier.java | 37 +++- .../internals/SerializedKeyValueIterator.java | 70 ++++++ .../state/internals/WindowStoreUtils.java | 3 + .../state/internals/WrappedStateStore.java | 90 ++++++++ .../internals/KGroupedStreamImplTest.java | 42 +++- .../internals/CachingSessionStoreTest.java | 3 +- .../internals/CachingWindowStoreTest.java | 5 +- .../ChangeLoggingKeyValueBytesStoreTest.java | 165 ++++++++++++++ .../ChangeLoggingKeyValueStoreTest.java | 207 ++++++++++++++++++ ...DelegatingPeekingKeyValueIteratorTest.java | 12 +- ...edSortedCacheSessionStoreIteratorTest.java | 113 ++++++++++ ...gedSortedCacheWindowStoreIteratorTest.java | 35 ++- .../internals/ReadOnlyWindowStoreStub.java | 5 + .../RocksDBKeyValueStoreSupplierTest.java | 155 +++++++++++++ .../RocksDBSessionStoreSupplierTest.java | 169 ++++++++++++++ .../RocksDBWindowStoreSupplierTest.java | 168 ++++++++++++++ .../internals/RocksDBWindowStoreTest.java | 2 - .../SerializedKeyValueIteratorTest.java | 95 ++++++++ 40 files changed, 2081 insertions(+), 642 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStore.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCachedWindowStoreIterator.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIterator.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedStateStore.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIteratorTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplierTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplierTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplierTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIteratorTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionKeySerde.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionKeySerde.java index 48213d67da153..d9a3528433916 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionKeySerde.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionKeySerde.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.kstream.Window; import org.apache.kafka.streams.kstream.Windowed; import java.nio.ByteBuffer; @@ -146,4 +147,20 @@ public static Bytes toBinary(final Windowed sessionKey, final Serializer< buf.putLong(sessionKey.window().start()); return new Bytes(buf.array()); } + + public static Bytes bytesToBinary(final Windowed sessionKey) { + final byte[] bytes = sessionKey.key().get(); + ByteBuffer buf = ByteBuffer.allocate(bytes.length + 2 * TIMESTAMP_SIZE); + buf.put(bytes); + buf.putLong(sessionKey.window().end()); + buf.putLong(sessionKey.window().start()); + return new Bytes(buf.array()); + } + + public static Window extractWindow(final byte [] binaryKey) { + final ByteBuffer buffer = ByteBuffer.wrap(binaryKey); + final long start = buffer.getLong(binaryKey.length - TIMESTAMP_SIZE); + final long end = buffer.getLong(binaryKey.length - 2 * TIMESTAMP_SIZE); + return new TimeWindow(start, end); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java index b6e6d0c2df381..958b7788411fa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java @@ -22,7 +22,6 @@ import org.apache.kafka.streams.KeyValue; import java.io.Closeable; -import java.util.Iterator; /** * Iterator interface of {@link KeyValue} with key typed {@link Long} used for {@link WindowStore#fetch(Object, long, long)}. @@ -32,7 +31,7 @@ * * @param Type of values */ -public interface WindowStoreIterator extends Iterator>, Closeable { +public interface WindowStoreIterator extends KeyValueIterator, Closeable { @Override void close(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java new file mode 100644 index 0000000000000..009dad05937c4 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java @@ -0,0 +1,166 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.kafka.streams.state.internals; + +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.StateSerdes; + +import java.util.NoSuchElementException; + +/** + * Merges two iterators. Assumes each of them is sorted by key + * + * @param + * @param + */ +abstract class AbstractMergedSortedCacheStoreIterator implements KeyValueIterator { + private final PeekingKeyValueIterator cacheIterator; + private final KeyValueIterator storeIterator; + protected final StateSerdes serdes; + + AbstractMergedSortedCacheStoreIterator(final PeekingKeyValueIterator cacheIterator, + final KeyValueIterator storeIterator, + final StateSerdes serdes) { + this.cacheIterator = cacheIterator; + this.storeIterator = storeIterator; + this.serdes = serdes; + } + + abstract int compare(final Bytes cacheKey, final KS storeKey); + + abstract K deserializeStoreKey(final KS key); + + abstract KeyValue deserializeStorePair(final KeyValue pair); + + abstract K deserializeCacheKey(final Bytes cacheKey); + + private boolean isDeletedCacheEntry(final KeyValue nextFromCache) { + return nextFromCache.value.value == null; + } + + @Override + public boolean hasNext() { + // skip over items deleted from cache, and corresponding store items if they have the same key + while (cacheIterator.hasNext() && isDeletedCacheEntry(cacheIterator.peekNext())) { + if (storeIterator.hasNext()) { + final KS nextStoreKey = storeIterator.peekNextKey(); + // advance the store iterator if the key is the same as the deleted cache key + if (compare(cacheIterator.peekNextKey(), nextStoreKey) == 0) { + storeIterator.next(); + } + } + cacheIterator.next(); + } + + return cacheIterator.hasNext() || storeIterator.hasNext(); + } + + @Override + public KeyValue next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + final Bytes nextCacheKey = cacheIterator.hasNext() ? cacheIterator.peekNextKey() : null; + final KS nextStoreKey = storeIterator.hasNext() ? storeIterator.peekNextKey() : null; + + if (nextCacheKey == null) { + return nextStoreValue(nextStoreKey); + } + + if (nextStoreKey == null) { + return nextCacheValue(nextCacheKey); + } + + final int comparison = compare(nextCacheKey, nextStoreKey); + if (comparison > 0) { + return nextStoreValue(nextStoreKey); + } else if (comparison < 0) { + return nextCacheValue(nextCacheKey); + } else { + // skip the same keyed element + storeIterator.next(); + return nextCacheValue(nextCacheKey); + } + } + + private KeyValue nextStoreValue(KS nextStoreKey) { + final KeyValue next = storeIterator.next(); + + if (!next.key.equals(nextStoreKey)) { + throw new IllegalStateException("Next record key is not the peeked key value; this should not happen"); + } + + return deserializeStorePair(next); + } + + private KeyValue nextCacheValue(Bytes nextCacheKey) { + final KeyValue next = cacheIterator.next(); + + if (!next.key.equals(nextCacheKey)) { + throw new IllegalStateException("Next record key is not the peeked key value; this should not happen"); + } + + return KeyValue.pair(deserializeCacheKey(next.key), serdes.valueFrom(next.value.value)); + } + + @Override + public K peekNextKey() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + final Bytes nextCacheKey = cacheIterator.hasNext() ? cacheIterator.peekNextKey() : null; + final KS nextStoreKey = storeIterator.hasNext() ? storeIterator.peekNextKey() : null; + + if (nextCacheKey == null) { + return deserializeStoreKey(nextStoreKey); + } + + if (nextStoreKey == null) { + return serdes.keyFrom(nextCacheKey.get()); + } + + final int comparison = compare(nextCacheKey, nextStoreKey); + if (comparison > 0) { + return deserializeStoreKey(nextStoreKey); + } else if (comparison < 0) { + return deserializeCacheKey(nextCacheKey); + } else { + // skip the same keyed element + storeIterator.next(); + return deserializeCacheKey(nextCacheKey); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException("remove() is not supported"); + } + + @Override + public void close() { + cacheIterator.close(); + storeIterator.close(); + } +} + diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java index fdb03fd13e0a5..9a0a9763cbfa1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java @@ -31,7 +31,7 @@ import java.util.List; -class CachingKeyValueStore implements KeyValueStore, CachedStateStore { +class CachingKeyValueStore implements WrappedStateStore, KeyValueStore, CachedStateStore { private final KeyValueStore underlying; private final Serde keySerde; @@ -234,4 +234,12 @@ public synchronized V delete(final K key) { KeyValueStore underlying() { return underlying; } + + @Override + public StateStore inner() { + if (underlying instanceof WrappedStateStore) { + return ((WrappedStateStore) underlying).inner(); + } + return underlying; + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java index 17c4ee0084684..fec660932a696 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.CacheFlushListener; import org.apache.kafka.streams.kstream.internals.SessionKeySerde; @@ -35,21 +34,22 @@ import java.util.NoSuchElementException; -class CachingSessionStore implements SessionStore, CachedStateStore, AGG> { +class CachingSessionStore extends WrappedStateStore.AbstractWrappedStateStore implements SessionStore, CachedStateStore, AGG> { - private final SegmentedBytesStore bytesStore; + private final SessionStore bytesStore; private final SessionKeySchema keySchema; private Serde keySerde; private final Serde aggSerde; private InternalProcessorContext context; private String name; - private StateSerdes, AGG> serdes; + private StateSerdes serdes; private ThreadCache cache; private CacheFlushListener, AGG> flushListener; - CachingSessionStore(final SegmentedBytesStore bytesStore, + CachingSessionStore(final SessionStore bytesStore, final Serde keySerde, final Serde aggSerde) { + super(bytesStore); this.bytesStore = bytesStore; this.keySerde = keySerde; this.aggSerde = aggSerde; @@ -65,12 +65,12 @@ public KeyValueIterator, AGG> findSessions(final K key, keySchema.lowerRange(binarySessionId, earliestSessionEndTime).get(), keySchema.upperRange(binarySessionId, latestSessionStartTime).get()); - final KeyValueIterator storeIterator = bytesStore.fetch(binarySessionId, earliestSessionEndTime, latestSessionStartTime); + final KeyValueIterator, byte[]> storeIterator = bytesStore.findSessions(binarySessionId, earliestSessionEndTime, latestSessionStartTime); final HasNextCondition hasNextCondition = keySchema.hasNextCondition(binarySessionId, - earliestSessionEndTime, - latestSessionStartTime); + earliestSessionEndTime, + latestSessionStartTime); final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition); - return new MergedSortedCacheKeyValueStoreIterator<>(filteredCacheIterator, storeIterator, serdes); + return new MergedSortedCacheSessionStoreIterator<>(filteredCacheIterator, storeIterator, serdes); } @@ -92,11 +92,6 @@ public KeyValueIterator, AGG> fetch(final K key) { return findSessions(key, 0, Long.MAX_VALUE); } - - public String name() { - return bytesStore.name(); - } - @SuppressWarnings("unchecked") public void init(final ProcessorContext context, final StateStore root) { bytesStore.init(context, root); @@ -107,14 +102,9 @@ public void init(final ProcessorContext context, final StateStore root) { private void initInternal(final InternalProcessorContext context) { this.context = context; - if (keySerde == null) { - keySerde = (Serde) context.keySerde(); - } - - - this.serdes = (StateSerdes, AGG>) new StateSerdes<>(bytesStore.name(), - new SessionKeySerde<>(keySerde), - aggSerde == null ? context.valueSerde() : aggSerde); + this.serdes = new StateSerdes<>(bytesStore.name(), + keySerde == null ? (Serde) context.keySerde() : keySerde, + aggSerde == null ? (Serde) context.valueSerde() : aggSerde); this.name = context.taskId() + "-" + bytesStore.name(); @@ -135,27 +125,27 @@ private void putAndMaybeForward(final ThreadCache.DirtyEntry entry, final Intern final RecordContext current = context.recordContext(); context.setRecordContext(entry.recordContext()); try { + final Windowed key = SessionKeySerde.from(binaryKey.get(), keySerde.deserializer()); if (flushListener != null) { - final Windowed key = SessionKeySerde.from(binaryKey.get(), keySerde.deserializer()); final AGG newValue = serdes.valueFrom(entry.newValue()); final AGG oldValue = fetchPrevious(binaryKey); if (!(newValue == null && oldValue == null)) { flushListener.apply(key, newValue == null ? null : newValue, oldValue); } - } - bytesStore.put(binaryKey, entry.newValue()); + bytesStore.put(new Windowed<>(Bytes.wrap(serdes.rawKey(key.key())), key.window()), entry.newValue()); } finally { context.setRecordContext(current); } } private AGG fetchPrevious(final Bytes key) { - final byte[] bytes = bytesStore.get(key); - if (bytes == null) { - return null; + try (final KeyValueIterator, byte[]> iterator = bytesStore.fetch(key)) { + if (!iterator.hasNext()) { + return null; + } + return serdes.valueFrom(iterator.next().value); } - return serdes.valueFrom(bytes); } @@ -170,25 +160,10 @@ public void close() { cache.close(name); } - public boolean persistent() { - return bytesStore.persistent(); - } - - public boolean isOpen() { - return bytesStore.isOpen(); - } - public void setFlushListener(CacheFlushListener, AGG> flushListener) { this.flushListener = flushListener; } - private void validateStoreOpen() { - if (!isOpen()) { - throw new InvalidStateStoreException("Store " + this.name + " is currently closed"); - } - } - - private static class FilteredCacheIterator implements PeekingKeyValueIterator { private final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator; private final HasNextCondition hasNextCondition; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index bd252f147d0ae..d471761f36475 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -17,8 +17,8 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.CacheFlushListener; import org.apache.kafka.streams.kstream.internals.TimeWindow; @@ -26,16 +26,15 @@ import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.RecordContext; -import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; import java.util.List; -class CachingWindowStore implements WindowStore, CachedStateStore, V> { +class CachingWindowStore extends WrappedStateStore.AbstractWrappedStateStore implements WindowStore, CachedStateStore, V> { - private final SegmentedBytesStore underlying; + private final WindowStore underlying; private final Serde keySerde; private final Serde valueSerde; private CacheFlushListener, V> flushListener; @@ -45,21 +44,17 @@ class CachingWindowStore implements WindowStore, CachedStateStore serdes; - CachingWindowStore(final SegmentedBytesStore underlying, + CachingWindowStore(final WindowStore underlying, final Serde keySerde, final Serde valueSerde, final long windowSize) { + super(underlying); this.underlying = underlying; this.keySerde = keySerde; this.valueSerde = valueSerde; this.windowSize = windowSize; } - @Override - public String name() { - return underlying.name(); - } - @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context, final StateStore root) { @@ -80,13 +75,14 @@ void initInternal(final ProcessorContext context) { @Override public void apply(final List entries) { for (ThreadCache.DirtyEntry entry : entries) { - final byte[] binaryKey = entry.key().get(); - final Bytes key = WindowStoreUtils.bytesKeyFromBinaryKey(binaryKey); - final long timestamp = WindowStoreUtils.timestampFromBinaryKey(binaryKey); - final Windowed windowedKey = new Windowed<>(WindowStoreUtils.keyFromBinaryKey(binaryKey, serdes), + final byte[] binaryWindowKey = entry.key().get(); + final long timestamp = WindowStoreUtils.timestampFromBinaryKey(binaryWindowKey); + + final Windowed windowedKey = new Windowed<>(WindowStoreUtils.keyFromBinaryKey(binaryWindowKey, serdes), new TimeWindow(timestamp, timestamp + windowSize)); - maybeForward(entry, Bytes.wrap(binaryKey), windowedKey, (InternalProcessorContext) context); - underlying.put(Bytes.wrap(WindowStoreUtils.toBinaryKey(key, timestamp, 0, WindowStoreUtils.INNER_SERDES)), entry.newValue()); + final Bytes key = WindowStoreUtils.bytesKeyFromBinaryKey(binaryWindowKey); + maybeForward(entry, key, windowedKey, (InternalProcessorContext) context); + underlying.put(key, entry.newValue(), timestamp); } } }); @@ -102,7 +98,7 @@ private void maybeForward(final ThreadCache.DirtyEntry entry, context.setRecordContext(entry.recordContext()); try { flushListener.apply(windowedKey, - serdes.valueFrom(entry.newValue()), fetchPrevious(key)); + serdes.valueFrom(entry.newValue()), fetchPrevious(key, windowedKey.window().start())); } finally { context.setRecordContext(current); } @@ -127,16 +123,6 @@ public void close() { cache.close(name); } - @Override - public boolean persistent() { - return underlying.persistent(); - } - - @Override - public boolean isOpen() { - return underlying.isOpen(); - } - @Override public synchronized void put(final K key, final V value) { put(key, value, context.timestamp()); @@ -158,23 +144,21 @@ public synchronized WindowStoreIterator fetch(final K key, final long timeFro byte[] binaryFrom = WindowStoreUtils.toBinaryKey(key, timeFrom, 0, serdes); byte[] binaryTo = WindowStoreUtils.toBinaryKey(key, timeTo, 0, serdes); - final KeyValueIterator underlyingIterator = underlying.fetch(Bytes.wrap(serdes.rawKey(key)), timeFrom, timeTo); + final WindowStoreIterator underlyingIterator = underlying.fetch(Bytes.wrap(serdes.rawKey(key)), timeFrom, timeTo); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(name, binaryFrom, binaryTo); - return new MergedSortedCachedWindowStoreIterator<>(cacheIterator, new DelegatingPeekingKeyValueIterator<>(name, underlyingIterator), serdes); + return new MergedSortedCacheWindowStoreIterator<>(cacheIterator, + underlyingIterator, + new StateSerdes<>(serdes.stateName(), Serdes.Long(), serdes.valueSerde())); } - private V fetchPrevious(final Bytes key) { - final byte[] result = underlying.get(key); - if (result == null) { - return null; - } - return serdes.valueFrom(result); - } - - private void validateStoreOpen() { - if (!isOpen()) { - throw new InvalidStateStoreException("Store " + this.name + " is currently closed"); + private V fetchPrevious(final Bytes key, final long timestamp) { + try (final WindowStoreIterator iter = underlying.fetch(key, timestamp, timestamp)) { + if (!iter.hasNext()) { + return null; + } else { + return serdes.valueFrom(iter.next().value); + } } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java new file mode 100644 index 0000000000000..e31d04bd1f51f --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java @@ -0,0 +1,93 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.state.internals; + +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; + +import java.util.List; + +public class ChangeLoggingKeyValueBytesStore extends WrappedStateStore.AbstractWrappedStateStore implements KeyValueStore { + private final KeyValueStore inner; + private StoreChangeLogger changeLogger; + + public ChangeLoggingKeyValueBytesStore(final KeyValueStore inner) { + super(inner); + this.inner = inner; + } + + @Override + public void init(final ProcessorContext context, final StateStore root) { + inner.init(context, root); + this.changeLogger = new StoreChangeLogger<>(inner.name(), context, WindowStoreUtils.INNER_SERDES); + } + + + @Override + public void put(final Bytes key, final byte[] value) { + inner.put(key, value); + changeLogger.logChange(key, value); + } + + @Override + public byte[] putIfAbsent(final Bytes key, final byte[] value) { + final byte[] previous = get(key); + if (previous == null) { + put(key, value); + } + return previous; + } + + @Override + public void putAll(final List> entries) { + inner.putAll(entries); + for (KeyValue entry : entries) { + changeLogger.logChange(entry.key, entry.value); + } + } + + @Override + public byte[] delete(final Bytes key) { + final byte[] oldValue = inner.get(key); + put(key, null); + return oldValue; + } + + @Override + public byte[] get(final Bytes key) { + return inner.get(key); + } + + @Override + public KeyValueIterator range(final Bytes from, final Bytes to) { + return inner.range(from, to); + } + + @Override + public KeyValueIterator all() { + return inner.all(); + } + + @Override + public long approximateNumEntries() { + return inner.approximateNumEntries(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStore.java new file mode 100644 index 0000000000000..11cf8022d8bb3 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStore.java @@ -0,0 +1,127 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.state.internals; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.StateSerdes; + +import java.util.ArrayList; +import java.util.List; + +class ChangeLoggingKeyValueStore extends WrappedStateStore.AbstractWrappedStateStore implements KeyValueStore { + private final ChangeLoggingKeyValueBytesStore innerBytes; + private final Serde keySerde; + private final Serde valueSerde; + private StateSerdes serdes; + + ChangeLoggingKeyValueStore(final KeyValueStore bytesStore, + final Serde keySerde, + final Serde valueSerde) { + this(new ChangeLoggingKeyValueBytesStore(bytesStore), keySerde, valueSerde); + } + + private ChangeLoggingKeyValueStore(final ChangeLoggingKeyValueBytesStore bytesStore, + final Serde keySerde, + final Serde valueSerde) { + super(bytesStore); + this.innerBytes = bytesStore; + this.keySerde = keySerde; + this.valueSerde = valueSerde; + } + + @Override + public String name() { + return null; + } + + @SuppressWarnings("unchecked") + @Override + public void init(final ProcessorContext context, final StateStore root) { + innerBytes.init(context, root); + + this.serdes = new StateSerdes<>(innerBytes.name(), + keySerde == null ? (Serde) context.keySerde() : keySerde, + valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + } + + @Override + public void put(final K key, final V value) { + final Bytes bytesKey = Bytes.wrap(serdes.rawKey(key)); + final byte[] bytesValue = serdes.rawValue(value); + innerBytes.put(bytesKey, bytesValue); + } + + @Override + public V putIfAbsent(final K key, final V value) { + final V v = get(key); + if (v == null) { + put(key, value); + } + return v; + } + + @Override + public void putAll(final List> entries) { + final List> keyValues = new ArrayList<>(); + for (final KeyValue entry : entries) { + keyValues.add(KeyValue.pair(Bytes.wrap(serdes.rawKey(entry.key)), serdes.rawValue(entry.value))); + } + innerBytes.putAll(keyValues); + } + + @Override + public V delete(final K key) { + final byte[] oldValue = innerBytes.delete(Bytes.wrap(serdes.rawKey(key))); + if (oldValue == null) { + return null; + } + return serdes.valueFrom(oldValue); + } + + @Override + public V get(final K key) { + final byte[] rawValue = innerBytes.get(Bytes.wrap(serdes.rawKey(key))); + if (rawValue == null) { + return null; + } + return serdes.valueFrom(rawValue); + } + + @Override + public KeyValueIterator range(final K from, final K to) { + return new SerializedKeyValueIterator<>(innerBytes.range(Bytes.wrap(serdes.rawKey(from)), + Bytes.wrap(serdes.rawKey(to))), + serdes); + } + + @Override + public KeyValueIterator all() { + return new SerializedKeyValueIterator<>(innerBytes.all(), serdes); + } + + @Override + public long approximateNumEntries() { + return innerBytes.approximateNumEntries(); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStore.java index 14b8f1710e287..21c28662522d5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStore.java @@ -25,13 +25,14 @@ * Simple wrapper around a {@link SegmentedBytesStore} to support writing * updates to a changelog */ -class ChangeLoggingSegmentedBytesStore implements SegmentedBytesStore { +class ChangeLoggingSegmentedBytesStore extends WrappedStateStore.AbstractWrappedStateStore implements SegmentedBytesStore { private final SegmentedBytesStore bytesStore; private StoreChangeLogger changeLogger; ChangeLoggingSegmentedBytesStore(final SegmentedBytesStore bytesStore) { + super(bytesStore); this.bytesStore = bytesStore; } @@ -60,10 +61,6 @@ public byte[] get(final Bytes key) { return bytesStore.get(key); } - @Override - public String name() { - return bytesStore.name(); - } @Override @SuppressWarnings("unchecked") @@ -71,25 +68,4 @@ public void init(final ProcessorContext context, final StateStore root) { bytesStore.init(context, root); changeLogger = new StoreChangeLogger<>(name(), context, WindowStoreUtils.INNER_SERDES); } - - @Override - public void flush() { - bytesStore.flush(); - } - - @Override - public void close() { - bytesStore.close(); - } - - @Override - public boolean persistent() { - return bytesStore.persistent(); - } - - @Override - public boolean isOpen() { - return bytesStore.isOpen(); - } - } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java index b33c0f0ceab3d..e0f1ec85b037b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java @@ -61,6 +61,11 @@ public WindowStoreIterator fetch(final K key, final long timeFrom, final long public void close() { } + @Override + public Long peekNextKey() { + throw new NoSuchElementException(); + } + @Override public boolean hasNext() { return false; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIterator.java index eb57acee688cf..f3101b17f9c54 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIterator.java @@ -22,7 +22,7 @@ import java.util.NoSuchElementException; -public class DelegatingPeekingKeyValueIterator implements KeyValueIterator { +public class DelegatingPeekingKeyValueIterator implements KeyValueIterator, PeekingKeyValueIterator { private final String storeName; private final KeyValueIterator underlying; private KeyValue next; @@ -78,4 +78,12 @@ public synchronized KeyValue next() { public void remove() { throw new UnsupportedOperationException("remove not supported"); } + + @Override + public KeyValue peekNext() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return next; + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueStoreIterator.java index c9a6866d9d17a..b860e161e0d9f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueStoreIterator.java @@ -21,141 +21,37 @@ import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.StateSerdes; -import java.util.Comparator; -import java.util.NoSuchElementException; - /** * Merges two iterators. Assumes each of them is sorted by key * * @param * @param */ -class MergedSortedCacheKeyValueStoreIterator implements KeyValueIterator { - private final PeekingKeyValueIterator cacheIterator; - private final KeyValueIterator storeIterator; - private final StateSerdes serdes; - private final Comparator comparator = Bytes.BYTES_LEXICO_COMPARATOR; - - public MergedSortedCacheKeyValueStoreIterator(final PeekingKeyValueIterator cacheIterator, - final KeyValueIterator storeIterator, - final StateSerdes serdes) { - this.cacheIterator = cacheIterator; - this.storeIterator = storeIterator; - this.serdes = serdes; - } - - @Override - public boolean hasNext() { - while (cacheIterator.hasNext() && isDeletedCacheEntry(cacheIterator.peekNext())) { - if (storeIterator.hasNext()) { - final Bytes storeKey = storeIterator.peekNextKey(); - // advance the store iterator if the key is the same as the deleted cache key - if (storeKey.equals(cacheIterator.peekNextKey())) { - storeIterator.next(); - } - } - // skip over items deleted from cache - cacheIterator.next(); - } - return cacheIterator.hasNext() || storeIterator.hasNext(); - } +class MergedSortedCacheKeyValueStoreIterator extends AbstractMergedSortedCacheStoreIterator { - - private boolean isDeletedCacheEntry(final KeyValue nextFromCache) { - return nextFromCache.value.value == null; + MergedSortedCacheKeyValueStoreIterator(final PeekingKeyValueIterator cacheIterator, + final KeyValueIterator storeIterator, + final StateSerdes serdes) { + super(cacheIterator, storeIterator, serdes); } - @Override - public KeyValue next() { - - return internalNext(new NextValueFunction>() { - @Override - public KeyValue apply(final byte[] cacheKey, final byte[] storeKey) { - if (cacheKey == null) { - return nextStoreValue(); - } - - if (storeKey == null) { - return nextCacheValue(); - } - - final int comparison = comparator.compare(cacheKey, storeKey); - if (comparison > 0) { - return nextStoreValue(); - } else if (comparison < 0) { - return nextCacheValue(); - } else { - storeIterator.next(); - return nextCacheValue(); - } - } - }); + public KeyValue deserializeStorePair(KeyValue pair) { + return KeyValue.pair(serdes.keyFrom(pair.key.get()), serdes.valueFrom(pair.value)); } @Override - public K peekNextKey() { - return internalNext(new NextValueFunction() { - @Override - public K apply(final byte[] cacheKey, final byte[] storeKey) { - if (cacheKey == null) { - return serdes.keyFrom(storeKey); - } - - if (storeKey == null) { - return serdes.keyFrom(cacheKey); - } - - final int comparison = comparator.compare(cacheKey, storeKey); - if (comparison > 0) { - return serdes.keyFrom(storeKey); - } else { - return serdes.keyFrom(cacheKey); - } - } - }); - } - - interface NextValueFunction { - T apply(final byte[] cacheKey, final byte [] storeKey); - } - - private T internalNext(final NextValueFunction nextValueFunction) { - if (!hasNext()) { - throw new NoSuchElementException(); - } - - byte[] nextCacheKey = null; - if (cacheIterator.hasNext()) { - nextCacheKey = cacheIterator.peekNextKey().get(); - } - - byte[] nextStoreKey = null; - if (storeIterator.hasNext()) { - nextStoreKey = storeIterator.peekNextKey().get(); - } - - return nextValueFunction.apply(nextCacheKey, nextStoreKey); - } - - private KeyValue nextCacheValue() { - final KeyValue next = cacheIterator.next(); - return KeyValue.pair(serdes.keyFrom(next.key.get()), serdes.valueFrom(next.value.value)); - } - - private KeyValue nextStoreValue() { - final KeyValue next = storeIterator.next(); - return KeyValue.pair(serdes.keyFrom(next.key.get()), serdes.valueFrom(next.value)); + K deserializeCacheKey(final Bytes cacheKey) { + return serdes.keyFrom(cacheKey.get()); } @Override - public void remove() { - throw new UnsupportedOperationException("remove not supported"); + public K deserializeStoreKey(Bytes key) { + return serdes.keyFrom(key.get()); } @Override - public void close() { - cacheIterator.close(); - storeIterator.close(); + public int compare(Bytes cacheKey, Bytes storeKey) { + return cacheKey.compareTo(storeKey); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java new file mode 100644 index 0000000000000..db64621340116 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java @@ -0,0 +1,71 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.kafka.streams.state.internals; + +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.internals.SessionKeySerde; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.StateSerdes; + +/** + * Merges two iterators. Assumes each of them is sorted by key + * + * @param + * @param + */ +class MergedSortedCacheSessionStoreIterator extends AbstractMergedSortedCacheStoreIterator, Windowed, AGG> { + private final StateSerdes rawSerdes; + + + MergedSortedCacheSessionStoreIterator(final PeekingKeyValueIterator cacheIterator, + final KeyValueIterator, byte[]> storeIterator, + final StateSerdes serdes) { + super(cacheIterator, storeIterator, new StateSerdes<>(serdes.stateName(), + new SessionKeySerde<>(serdes.keySerde()), + serdes.valueSerde())); + + rawSerdes = serdes; + } + + @Override + public KeyValue, AGG> deserializeStorePair(KeyValue, byte[]> pair) { + final K key = rawSerdes.keyFrom(pair.key.key().get()); + return KeyValue.pair(new Windowed<>(key, pair.key.window()), serdes.valueFrom(pair.value)); + } + + @Override + Windowed deserializeCacheKey(final Bytes cacheKey) { + return SessionKeySerde.from(cacheKey.get(), rawSerdes.keyDeserializer()); + } + + @Override + public Windowed deserializeStoreKey(Windowed key) { + final K originalKey = rawSerdes.keyFrom(key.key().get()); + return new Windowed(originalKey, key.window()); + } + + @Override + public int compare(Bytes cacheKey, Windowed storeKey) { + Bytes storeKeyBytes = SessionKeySerde.bytesToBinary(storeKey); + return cacheKey.compareTo(storeKeyBytes); + } +} + diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java new file mode 100644 index 0000000000000..a9d097320086c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.state.internals; + +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.streams.state.WindowStoreIterator; + +/** + * Merges two iterators. Assumes each of them is sorted by key + * + * @param + */ +class MergedSortedCacheWindowStoreIterator extends AbstractMergedSortedCacheStoreIterator implements WindowStoreIterator { + + MergedSortedCacheWindowStoreIterator(final PeekingKeyValueIterator cacheIterator, + final KeyValueIterator storeIterator, + final StateSerdes serdes) { + super(cacheIterator, storeIterator, serdes); + } + + @Override + public KeyValue deserializeStorePair(final KeyValue pair) { + return KeyValue.pair(pair.key, serdes.valueFrom(pair.value)); + } + + @Override + Long deserializeCacheKey(final Bytes cacheKey) { + return WindowStoreUtils.timestampFromBinaryKey(cacheKey.get()); + } + + @Override + public Long deserializeStoreKey(final Long key) { + return key; + } + + @Override + public int compare(final Bytes cacheKey, final Long storeKey) { + final Long cacheTimestamp = WindowStoreUtils.timestampFromBinaryKey(cacheKey.get()); + return cacheTimestamp.compareTo(storeKey); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCachedWindowStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCachedWindowStoreIterator.java deleted file mode 100644 index e210e73111fe9..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCachedWindowStoreIterator.java +++ /dev/null @@ -1,107 +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.kafka.streams.state.internals; - -import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.state.KeyValueIterator; -import org.apache.kafka.streams.state.StateSerdes; -import org.apache.kafka.streams.state.WindowStoreIterator; - -import java.util.NoSuchElementException; - -/** - * Merges two iterators. Assumes each of them is sorted by key - * - * @param - * @param - */ -class MergedSortedCachedWindowStoreIterator implements WindowStoreIterator { - private final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator; - private final KeyValueIterator storeIterator; - private final StateSerdes serdes; - - public MergedSortedCachedWindowStoreIterator(final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator, - final KeyValueIterator storeIterator, - final StateSerdes serdes) { - this.cacheIterator = cacheIterator; - this.storeIterator = storeIterator; - this.serdes = serdes; - } - - @Override - public boolean hasNext() { - return cacheIterator.hasNext() || storeIterator.hasNext(); - } - - - @Override - public KeyValue next() { - if (!hasNext()) { - throw new NoSuchElementException(); - } - - Long nextCacheTimestamp = null; - if (cacheIterator.hasNext()) { - nextCacheTimestamp = WindowStoreUtils.timestampFromBinaryKey(cacheIterator.peekNextKey().get()); - } - - Long nextStoreTimestamp = null; - if (storeIterator.hasNext()) { - nextStoreTimestamp = WindowStoreUtils.timestampFromBinaryKey(storeIterator.peekNextKey().get()); - } - - if (nextCacheTimestamp == null) { - return nextStoreValue(nextStoreTimestamp); - } - - if (nextStoreTimestamp == null) { - return nextCacheValue(nextCacheTimestamp); - } - - final int comparison = nextCacheTimestamp.compareTo(nextStoreTimestamp); - if (comparison > 0) { - return nextStoreValue(nextStoreTimestamp); - } else if (comparison < 0) { - return nextCacheValue(nextCacheTimestamp); - } else { - storeIterator.next(); - return nextCacheValue(nextCacheTimestamp); - } - } - - private KeyValue nextCacheValue(final Long timestamp) { - final KeyValue next = cacheIterator.next(); - return KeyValue.pair(timestamp, serdes.valueFrom(next.value.value)); - } - - private KeyValue nextStoreValue(final Long timestamp) { - final KeyValue next = storeIterator.next(); - return KeyValue.pair(timestamp, serdes.valueFrom(next.value)); - } - - @Override - public void remove() { - throw new UnsupportedOperationException("remove not supported"); - } - - @Override - public void close() { - cacheIterator.close(); - storeIterator.close(); - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java index 926e5d4a4acf0..7dc2d3350dd8a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java @@ -35,7 +35,7 @@ * @param * @param */ -public class MeteredKeyValueStore implements KeyValueStore { +public class MeteredKeyValueStore extends WrappedStateStore.AbstractWrappedStateStore implements KeyValueStore { protected final KeyValueStore inner; protected final String metricScope; @@ -102,17 +102,15 @@ public void run() { }; // always wrap the store with the metered store - public MeteredKeyValueStore(final KeyValueStore inner, String metricScope, Time time) { + public MeteredKeyValueStore(final KeyValueStore inner, + final String metricScope, + final Time time) { + super(inner); this.inner = inner; this.metricScope = metricScope; this.time = time != null ? time : Time.SYSTEM; } - @Override - public String name() { - return inner.name(); - } - @Override public void init(ProcessorContext context, StateStore root) { final String name = name(); @@ -133,16 +131,6 @@ public void init(ProcessorContext context, StateStore root) { metrics.measureLatencyNs(time, initDelegate, this.restoreTime); } - @Override - public boolean persistent() { - return inner.persistent(); - } - - @Override - public boolean isOpen() { - return inner.isOpen(); - } - @Override public V get(K key) { this.key = key; @@ -193,11 +181,6 @@ public long approximateNumEntries() { return this.inner.approximateNumEntries(); } - @Override - public void close() { - inner.close(); - } - @Override public void flush() { metrics.measureLatencyNs(time, flushDelegate, this.flushTime); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStore.java index e0ed03ed06cd9..4eb3936be60e0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStore.java @@ -27,7 +27,7 @@ import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueIterator; -class MeteredSegmentedBytesStore implements SegmentedBytesStore { +class MeteredSegmentedBytesStore extends WrappedStateStore.AbstractWrappedStateStore implements SegmentedBytesStore { private final SegmentedBytesStore inner; private final String metricScope; @@ -40,17 +40,15 @@ class MeteredSegmentedBytesStore implements SegmentedBytesStore { private Sensor getTime; private Sensor removeTime; - MeteredSegmentedBytesStore(final SegmentedBytesStore inner, String metricScope, Time time) { + MeteredSegmentedBytesStore(final SegmentedBytesStore inner, + final String metricScope, + final Time time) { + super(inner); this.inner = inner; this.metricScope = metricScope; this.time = time != null ? time : new SystemTime(); } - @Override - public String name() { - return inner.name(); - } - @Override public void init(ProcessorContext context, StateStore root) { final String name = name(); @@ -71,16 +69,6 @@ public void init(ProcessorContext context, StateStore root) { } } - @Override - public boolean persistent() { - return inner.persistent(); - } - - @Override - public boolean isOpen() { - return inner.isOpen(); - } - @Override public byte[] get(final Bytes key) { final long startNs = time.nanoseconds(); @@ -116,11 +104,6 @@ public void put(final Bytes key, final byte[] value) { } } - @Override - public void close() { - inner.close(); - } - @Override public void flush() { final long startNs = time.nanoseconds(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java deleted file mode 100644 index c725c1ac6cefd..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ /dev/null @@ -1,180 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streams.state.internals; - -import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.internals.StreamsMetricsImpl; -import org.apache.kafka.streams.state.WindowStore; -import org.apache.kafka.streams.state.WindowStoreIterator; - -public class MeteredWindowStore implements WindowStore { - - protected final WindowStore inner; - protected final String metricScope; - protected final Time time; - - private Sensor putTime; - private Sensor fetchTime; - private Sensor flushTime; - private Sensor restoreTime; - private StreamsMetricsImpl metrics; - - private ProcessorContext context; - private StateStore root; - private Runnable initDelegate = new Runnable() { - @Override - public void run() { - inner.init(context, root); - } - }; - - private K key; - private V value; - private long timestamp; - private Runnable putDelegate = new Runnable() { - @Override - public void run() { - inner.put(key, value); - } - }; - private Runnable putTsDelegate = new Runnable() { - @Override - public void run() { - inner.put(key, value, timestamp); - } - }; - private Runnable flushDelegate = new Runnable() { - @Override - public void run() { - inner.flush(); - } - }; - - // always wrap the store with the metered store - public MeteredWindowStore(final WindowStore inner, String metricScope, Time time) { - this.inner = inner; - this.metricScope = metricScope; - this.time = time != null ? time : Time.SYSTEM; - } - - @Override - public String name() { - return inner.name(); - } - - @Override - public void init(ProcessorContext context, StateStore root) { - final String name = name(); - this.context = context; - this.root = root; - this.metrics = (StreamsMetricsImpl) context.metrics(); - this.putTime = this.metrics.addLatencySensor(metricScope, name, "put", Sensor.RecordingLevel.DEBUG); - this.fetchTime = this.metrics.addLatencySensor(metricScope, name, "fetch", Sensor.RecordingLevel.DEBUG); - this.flushTime = this.metrics.addLatencySensor(metricScope, name, "flush", Sensor.RecordingLevel.DEBUG); - this.restoreTime = this.metrics.addLatencySensor(metricScope, name, "restore", Sensor.RecordingLevel.DEBUG); - - // register and possibly restore the state from the logs - metrics.measureLatencyNs(time, initDelegate, this.restoreTime); - } - - @Override - public boolean persistent() { - return inner.persistent(); - } - - @Override - public boolean isOpen() { - return inner.isOpen(); - } - - @Override - public WindowStoreIterator fetch(K key, long timeFrom, long timeTo) { - return new MeteredWindowStoreIterator<>(this.inner.fetch(key, timeFrom, timeTo), this.fetchTime); - } - - @Override - public void put(K key, V value) { - this.key = key; - this.value = value; - metrics.measureLatencyNs(time, putDelegate, this.putTime); - } - - @Override - public void put(K key, V value, long timestamp) { - this.key = key; - this.value = value; - this.timestamp = timestamp; - metrics.measureLatencyNs(time, putTsDelegate, this.putTime); - } - - @Override - public void close() { - inner.close(); - } - - @Override - public void flush() { - metrics.measureLatencyNs(time, flushDelegate, this.flushTime); - } - - private class MeteredWindowStoreIterator implements WindowStoreIterator { - - private final WindowStoreIterator iter; - private final Sensor sensor; - private final long startNs; - - public MeteredWindowStoreIterator(WindowStoreIterator iter, Sensor sensor) { - this.iter = iter; - this.sensor = sensor; - this.startNs = time.nanoseconds(); - } - - @Override - public boolean hasNext() { - return iter.hasNext(); - } - - @Override - public KeyValue next() { - return iter.next(); - } - - @Override - public void remove() { - iter.remove(); - } - - @Override - public void close() { - try { - iter.close(); - } finally { - metrics.recordLatency(this.sensor, this.startNs, time.nanoseconds()); - } - } - - } - - WindowStore inner() { - return inner; - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java index 164b3525d3701..b72bbede4c52b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java @@ -35,28 +35,54 @@ public class RocksDBKeyValueStoreSupplier extends AbstractStoreSupplier { - private final boolean enableCaching; + private static final String METRICS_SCOPE = "rocksdb-state"; + private final boolean cached; - public RocksDBKeyValueStoreSupplier(String name, Serde keySerde, Serde valueSerde, boolean logged, Map logConfig, boolean enableCaching) { - this(name, keySerde, valueSerde, null, logged, logConfig, enableCaching); + public RocksDBKeyValueStoreSupplier(String name, Serde keySerde, Serde valueSerde, boolean logged, Map logConfig, boolean cached) { + this(name, keySerde, valueSerde, null, logged, logConfig, cached); } - public RocksDBKeyValueStoreSupplier(String name, Serde keySerde, Serde valueSerde, Time time, boolean logged, Map logConfig, boolean enableCaching) { + public RocksDBKeyValueStoreSupplier(String name, Serde keySerde, Serde valueSerde, Time time, boolean logged, Map logConfig, boolean cached) { super(name, keySerde, valueSerde, time, logged, logConfig); - this.enableCaching = enableCaching; + this.cached = cached; } public KeyValueStore get() { - if (!enableCaching) { - RocksDBStore store = new RocksDBStore<>(name, keySerde, valueSerde); - return new MeteredKeyValueStore<>(logged ? store.enableLogging() : store, "rocksdb-state", time); + if (!cached && !logged) { + return new MeteredKeyValueStore<>( + new RocksDBStore<>(name, keySerde, valueSerde), METRICS_SCOPE, time); + } + + // when cached, logged, or both we use a bytes store as the inner most store + final RocksDBStore rocks = new RocksDBStore<>(name, + Serdes.Bytes(), + Serdes.ByteArray()); + + if (cached && logged) { + return new CachingKeyValueStore<>( + new MeteredKeyValueStore<>( + new ChangeLoggingKeyValueBytesStore(rocks), + METRICS_SCOPE, + time), + keySerde, + valueSerde); + } + + if (cached) { + return new CachingKeyValueStore<>( + new MeteredKeyValueStore<>(rocks, METRICS_SCOPE, time), + keySerde, + valueSerde); + + } else { + // logged + return new MeteredKeyValueStore<>( + new ChangeLoggingKeyValueStore<>(rocks, keySerde, valueSerde), + METRICS_SCOPE, + time); } - final RocksDBStore store = new RocksDBStore<>(name, Serdes.Bytes(), Serdes.ByteArray()); - return new CachingKeyValueStore<>(new MeteredKeyValueStore<>(logged ? store.enableLogging() : store, - "rocksdb-state", - time), - keySerde, - valueSerde); } + + } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplier.java index 76454724ba7fd..10ebf65b217e8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplier.java @@ -5,9 +5,9 @@ * 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 - * + *

    + * 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. @@ -18,6 +18,8 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.state.SessionStore; @@ -35,13 +37,14 @@ public class RocksDBSessionStoreSupplier extends AbstractStoreSupplier implements WindowStoreSupplier { private static final int NUM_SEGMENTS = 3; + public static final String METRIC_SCOPE = "rocksdb-session-store"; private final long retentionPeriod; - private final boolean enableCaching; + private final boolean cached; - public RocksDBSessionStoreSupplier(String name, long retentionPeriod, Serde keySerde, Serde valueSerde, boolean logged, Map logConfig, boolean enableCaching) { + public RocksDBSessionStoreSupplier(String name, long retentionPeriod, Serde keySerde, Serde valueSerde, boolean logged, Map logConfig, boolean cached) { super(name, keySerde, valueSerde, Time.SYSTEM, logged, logConfig); this.retentionPeriod = retentionPeriod; - this.enableCaching = enableCaching; + this.cached = cached; } public String name() { @@ -49,16 +52,41 @@ public String name() { } public SessionStore get() { - final RocksDBSegmentedBytesStore bytesStore = new RocksDBSegmentedBytesStore(name, + final SessionKeySchema keySchema = new SessionKeySchema(); + final RocksDBSegmentedBytesStore segmented = new RocksDBSegmentedBytesStore(name, retentionPeriod, NUM_SEGMENTS, - new SessionKeySchema()); - final MeteredSegmentedBytesStore metered = new MeteredSegmentedBytesStore(logged ? new ChangeLoggingSegmentedBytesStore(bytesStore) - : bytesStore, "rocksdb-session-store", time); - if (enableCaching) { - return new CachingSessionStore<>(metered, keySerde, valueSerde); + keySchema + ); + + if (cached && logged) { + final ChangeLoggingSegmentedBytesStore logged = new ChangeLoggingSegmentedBytesStore(segmented); + final MeteredSegmentedBytesStore metered = new MeteredSegmentedBytesStore(logged, + METRIC_SCOPE, time); + final RocksDBSessionStore sessionStore + = new RocksDBSessionStore<>(metered, Serdes.Bytes(), Serdes.ByteArray()); + + return new CachingSessionStore<>(sessionStore, keySerde, valueSerde); + } + + if (cached) { + final MeteredSegmentedBytesStore metered = new MeteredSegmentedBytesStore(segmented, + METRIC_SCOPE, time); + final RocksDBSessionStore sessionStore + = new RocksDBSessionStore<>(metered, Serdes.Bytes(), Serdes.ByteArray()); + + return new CachingSessionStore<>(sessionStore, keySerde, valueSerde); } - return new RocksDBSessionStore<>(metered, keySerde, valueSerde); + + if (logged) { + final ChangeLoggingSegmentedBytesStore logged = new ChangeLoggingSegmentedBytesStore(segmented); + final MeteredSegmentedBytesStore metered = new MeteredSegmentedBytesStore(logged, + METRIC_SCOPE, time); + return new RocksDBSessionStore<>(metered, keySerde, valueSerde); + } + + return new RocksDBSessionStore<>( + new MeteredSegmentedBytesStore(segmented, METRIC_SCOPE, time), keySerde, valueSerde); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index 8b838d02dc2f2..3f8d509dda05a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -92,24 +92,20 @@ public class RocksDBStore implements KeyValueStore { private WriteOptions wOptions; private FlushOptions fOptions; - private boolean loggingEnabled = false; - - private StoreChangeLogger changeLogger; - protected volatile boolean open = false; - public KeyValueStore enableLogging() { - loggingEnabled = true; - - return this; - } - public RocksDBStore(String name, Serde keySerde, Serde valueSerde) { + public RocksDBStore(final String name, + final Serde keySerde, + final Serde valueSerde) { this(name, DB_FILE_DIR, keySerde, valueSerde); } - public RocksDBStore(String name, String parentDir, Serde keySerde, Serde valueSerde) { + public RocksDBStore(final String name, + final String parentDir, + final Serde keySerde, + final Serde valueSerde) { this.name = name; this.parentDir = parentDir; this.keySerde = keySerde; @@ -159,10 +155,9 @@ public void init(ProcessorContext context, StateStore root) { // open the DB dir openDB(context); - this.changeLogger = this.loggingEnabled ? new StoreChangeLogger<>(name, context, WindowStoreUtils.INNER_SERDES) : null; // value getter should always read directly from rocksDB // since it is only for values that are already flushed - context.register(root, loggingEnabled, new StateRestoreCallback() { + context.register(root, false, new StateRestoreCallback() { @Override public void restore(byte[] key, byte[] value) { @@ -235,10 +230,6 @@ public synchronized void put(K key, V value) { byte[] rawKey = serdes.rawKey(key); byte[] rawValue = serdes.rawValue(value); putInternal(rawKey, rawValue); - - if (loggingEnabled) { - changeLogger.logChange(Bytes.wrap(rawKey), rawValue); - } } @Override @@ -278,9 +269,6 @@ public void putAll(List> entries) { } else { final byte[] value = serdes.rawValue(entry.value); batch.put(rawKey, value); - if (loggingEnabled) { - changeLogger.logChange(Bytes.wrap(rawKey), value); - } } } db.write(wOptions, batch); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index a2a420e1c3363..80c4796a6cafc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -20,6 +20,7 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorContext; @@ -33,7 +34,6 @@ class RocksDBWindowStore implements WindowStore { - private final String name; private final SegmentedBytesStore bytesStore; private final boolean retainDuplicates; private final Serde keySerde; @@ -43,8 +43,15 @@ class RocksDBWindowStore implements WindowStore { private StateSerdes serdes; - RocksDBWindowStore(String name, boolean retainDuplicates, Serde keySerde, Serde valueSerde, final SegmentedBytesStore bytesStore) { - this.name = name; + static RocksDBWindowStore bytesStore(final SegmentedBytesStore inner, final boolean retainDuplicates) { + return new RocksDBWindowStore<>(inner, Serdes.Bytes(), Serdes.ByteArray(), retainDuplicates); + } + + + RocksDBWindowStore(final SegmentedBytesStore bytesStore, + final Serde keySerde, + final Serde valueSerde, + final boolean retainDuplicates) { this.keySerde = keySerde; this.valueSerde = valueSerde; this.retainDuplicates = retainDuplicates; @@ -54,7 +61,7 @@ class RocksDBWindowStore implements WindowStore { @Override public String name() { - return name; + return bytesStore.name(); } @Override @@ -62,7 +69,7 @@ public String name() { public void init(final ProcessorContext context, final StateStore root) { this.context = context; // construct the serde - this.serdes = new StateSerdes<>(name, + this.serdes = new StateSerdes<>(bytesStore.name(), keySerde == null ? (Serde) context.keySerde() : keySerde, valueSerde == null ? (Serde) context.valueSerde() : valueSerde); @@ -147,6 +154,14 @@ public void remove() { public void close() { actual.close(); } + + @Override + public Long peekNextKey() { + if (!actual.hasNext()) { + throw new NoSuchElementException(); + } + return WindowStoreUtils.timestampFromBinaryKey(actual.peekNextKey().get()); + } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java index 84f1734fb7c8e..abaaffd2bff84 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.state.WindowStore; @@ -58,23 +59,35 @@ public String name() { } public WindowStore get() { - final RocksDBSegmentedBytesStore bytesStore = new RocksDBSegmentedBytesStore(name, retentionPeriod, numSegments, new WindowStoreKeySchema()); - if (!enableCaching) { - final RocksDBWindowStore segmentedStore = new RocksDBWindowStore<>(name, retainDuplicates, keySerde, valueSerde, - logged ? new ChangeLoggingSegmentedBytesStore(bytesStore) - : bytesStore); - return new MeteredWindowStore<>(segmentedStore, "rocksdb-window", time); - } + return maybeWrapCaching( + maybeWrapLogged( + new RocksDBSegmentedBytesStore( + name, + retentionPeriod, + numSegments, + new WindowStoreKeySchema() + ))); - return new CachingWindowStore<>(new MeteredSegmentedBytesStore(logged ? new ChangeLoggingSegmentedBytesStore(bytesStore) - : bytesStore, - "rocksdb-window", - time), - keySerde, valueSerde, windowSize); } @Override public long retentionPeriod() { return retentionPeriod; } + + private SegmentedBytesStore maybeWrapLogged(final SegmentedBytesStore inner) { + if (!logged) { + return inner; + } + return new ChangeLoggingSegmentedBytesStore(inner); + } + + private WindowStore maybeWrapCaching(final SegmentedBytesStore inner) { + final MeteredSegmentedBytesStore metered = new MeteredSegmentedBytesStore(inner, "rocksdb-window", time); + if (!enableCaching) { + return new RocksDBWindowStore<>(metered, keySerde, valueSerde, retainDuplicates); + } + final RocksDBWindowStore windowed = RocksDBWindowStore.bytesStore(metered, retainDuplicates); + return new CachingWindowStore<>(windowed, keySerde, valueSerde, windowSize); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIterator.java new file mode 100644 index 0000000000000..d76e8a4568e7b --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIterator.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.state.internals; + +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.StateSerdes; + +import java.util.NoSuchElementException; + +class SerializedKeyValueIterator implements KeyValueIterator { + + private final KeyValueIterator bytesIterator; + private final StateSerdes serdes; + + SerializedKeyValueIterator(final KeyValueIterator bytesIterator, + final StateSerdes serdes) { + + this.bytesIterator = bytesIterator; + this.serdes = serdes; + } + + @Override + public void close() { + bytesIterator.close(); + } + + @Override + public K peekNextKey() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + final Bytes bytes = bytesIterator.peekNextKey(); + return serdes.keyFrom(bytes.get()); + } + + @Override + public boolean hasNext() { + return bytesIterator.hasNext(); + } + + @Override + public KeyValue next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + final KeyValue next = bytesIterator.next(); + return KeyValue.pair(serdes.keyFrom(next.key.get()), serdes.valueFrom(next.value)); + } + + @Override + public void remove() { + throw new UnsupportedOperationException("remove not supported by SerializedKeyValueIterator"); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java index 1ea6bef7df4aa..074cf8aef5443 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java @@ -39,7 +39,10 @@ public class WindowStoreUtils { public static byte[] toBinaryKey(K key, final long timestamp, final int seqnum, StateSerdes serdes) { byte[] serializedKey = serdes.rawKey(key); + return toBinaryKey(serializedKey, timestamp, seqnum); + } + static byte[] toBinaryKey(byte[] serializedKey, final long timestamp, final int seqnum) { ByteBuffer buf = ByteBuffer.allocate(serializedKey.length + TIMESTAMP_SIZE + SEQNUM_SIZE); buf.put(serializedKey); buf.putLong(timestamp); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedStateStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedStateStore.java new file mode 100644 index 0000000000000..3d80b98d69744 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedStateStore.java @@ -0,0 +1,90 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.kafka.streams.state.internals; + +import org.apache.kafka.streams.errors.InvalidStateStoreException; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; + +/** + * A storage engine wrapper for utilities like logging, caching, and metering. + */ +interface WrappedStateStore extends StateStore { + + /** + * Return the inner storage engine + * + * @return wrapped inner storage engine + */ + StateStore inner(); + + abstract class AbstractWrappedStateStore implements WrappedStateStore { + final StateStore innerState; + + AbstractWrappedStateStore(StateStore inner) { + this.innerState = inner; + } + + @Override + public void init(ProcessorContext context, StateStore root) { + innerState.init(context, root); + } + + @Override + public String name() { + return innerState.name(); + } + + @Override + public boolean persistent() { + return innerState.persistent(); + } + + @Override + public boolean isOpen() { + return innerState.isOpen(); + } + + void validateStoreOpen() { + if (!innerState.isOpen()) { + throw new InvalidStateStoreException("Store " + innerState.name() + " is currently closed."); + } + } + + @Override + public StateStore inner() { + if (innerState instanceof WrappedStateStore) { + return ((WrappedStateStore) innerState).inner(); + } + return innerState; + } + + @Override + public void flush() { + innerState.flush(); + } + + @Override + public void close() { + innerState.close(); + } + } + + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java index 729e1905b43d3..b6d8a974200f3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.ForeachAction; import org.apache.kafka.streams.kstream.Initializer; @@ -40,9 +41,14 @@ import org.junit.Before; import org.junit.Test; +import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; public class KGroupedStreamImplTest { @@ -337,4 +343,38 @@ public void shouldNotAcceptNullStoreNameWhenCountingSessionWindows() throws Exce public void shouldNotAcceptNullStoreStoreSupplierNameWhenCountingSessionWindows() throws Exception { groupedStream.count(SessionWindows.with(90), (StateStoreSupplier) null); } -} + + @Test + public void shouldCountWindowed() throws Exception { + final List, Long>> results = new ArrayList<>(); + groupedStream.count( + TimeWindows.of(500L), + "aggregate-by-key-windowed") + .foreach(new ForeachAction, Long>() { + @Override + public void apply(final Windowed key, final Long value) { + results.add(KeyValue.pair(key, value)); + } + }); + + final KStreamTestDriver driver = new KStreamTestDriver(builder, TestUtils.tempDirectory(), 0); + driver.setTime(0); + driver.process(TOPIC, "1", "A"); + driver.process(TOPIC, "2", "B"); + driver.process(TOPIC, "3", "C"); + driver.setTime(500); + driver.process(TOPIC, "1", "A"); + driver.process(TOPIC, "1", "A"); + driver.process(TOPIC, "2", "B"); + driver.process(TOPIC, "2", "B"); + assertThat(results, equalTo(Arrays.asList( + KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), 1L), + KeyValue.pair(new Windowed<>("2", new TimeWindow(0, 500)), 1L), + KeyValue.pair(new Windowed<>("3", new TimeWindow(0, 500)), 1L), + KeyValue.pair(new Windowed<>("1", new TimeWindow(500, 1000)), 1L), + KeyValue.pair(new Windowed<>("1", new TimeWindow(500, 1000)), 2L), + KeyValue.pair(new Windowed<>("2", new TimeWindow(500, 1000)), 1L), + KeyValue.pair(new Windowed<>("2", new TimeWindow(500, 1000)), 2L) + ))); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java index c603aa0b93fcf..5035f70887e8d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java @@ -59,7 +59,8 @@ public class CachingSessionStoreTest { @Before public void setUp() throws Exception { underlying = new RocksDBSegmentedBytesStore("test", 60000, 3, new SessionKeySchema()); - cachingStore = new CachingSessionStore<>(underlying, + final RocksDBSessionStore sessionStore = new RocksDBSessionStore<>(underlying, Serdes.Bytes(), Serdes.ByteArray()); + cachingStore = new CachingSessionStore<>(sessionStore, Serdes.String(), Serdes.Long()); cache = new ThreadCache("testCache", MAX_CACHE_SIZE_BYTES, new MockStreamsMetrics(new Metrics())); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java index 37fc9a0f6a69e..1de1002d74c51 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java @@ -54,13 +54,15 @@ public class CachingWindowStoreTest { private String topic; private static final long DEFAULT_TIMESTAMP = 10L; private WindowStoreKeySchema keySchema; + private RocksDBWindowStore windowStore; @Before public void setUp() throws Exception { keySchema = new WindowStoreKeySchema(); underlying = new RocksDBSegmentedBytesStore("test", 30000, 3, keySchema); + windowStore = new RocksDBWindowStore<>(underlying, Serdes.Bytes(), Serdes.ByteArray(), false); cacheListener = new CachingKeyValueStoreTest.CacheFlushListenerStub<>(); - cachingStore = new CachingWindowStore<>(underlying, + cachingStore = new CachingWindowStore<>(windowStore, Serdes.String(), Serdes.String(), WINDOW_SIZE); @@ -72,6 +74,7 @@ public void setUp() throws Exception { cachingStore.init(context, cachingStore); } + @Test public void shouldPutFetchFromCache() throws Exception { cachingStore.put("a", "a"); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java new file mode 100644 index 0000000000000..82fb83123a095 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java @@ -0,0 +1,165 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.state.internals; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; +import org.apache.kafka.test.InMemoryKeyValueStore; +import org.apache.kafka.test.MockProcessorContext; +import org.apache.kafka.test.NoOpRecordCollector; +import org.apache.kafka.test.TestUtils; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; + +public class ChangeLoggingKeyValueBytesStoreTest { + + private final InMemoryKeyValueStore inner = new InMemoryKeyValueStore<>("kv"); + private final ChangeLoggingKeyValueBytesStore store = new ChangeLoggingKeyValueBytesStore(inner); + private final Map sent = new HashMap<>(); + private final Bytes hi = Bytes.wrap("hi".getBytes()); + private final Bytes hello = Bytes.wrap("hello".getBytes()); + private final byte[] there = "there".getBytes(); + private final byte[] world = "world".getBytes(); + + @Before + public void before() { + final NoOpRecordCollector collector = new NoOpRecordCollector() { + @Override + public void send(final ProducerRecord record, final Serializer keySerializer, final Serializer valueSerializer) { + sent.put(record.key(), record.value()); + } + }; + final MockProcessorContext context = new MockProcessorContext(null, + TestUtils.tempDirectory(), + Serdes.String(), + Serdes.Long(), + collector, + new ThreadCache("testCache", 0, new MockStreamsMetrics(new Metrics()))); + context.setTime(0); + store.init(context, store); + } + + @Test + public void shouldWriteKeyValueBytesToInnerStoreOnPut() throws Exception { + store.put(hi, there); + assertThat(inner.get(hi), equalTo(there)); + } + + @Test + public void shouldLogChangeOnPut() throws Exception { + store.put(hi, there); + assertThat((byte[]) sent.get(hi), equalTo(there)); + } + + @Test + public void shouldWriteAllKeyValueToInnerStoreOnPutAll() throws Exception { + store.putAll(Arrays.asList(KeyValue.pair(hi, there), + KeyValue.pair(hello, world))); + assertThat(inner.get(hi), equalTo(there)); + assertThat(inner.get(hello), equalTo(world)); + } + + @Test + public void shouldLogChangesOnPutAll() throws Exception { + store.putAll(Arrays.asList(KeyValue.pair(hi, there), + KeyValue.pair(hello, world))); + assertThat((byte[]) sent.get(hi), equalTo(there)); + assertThat((byte[]) sent.get(hello), equalTo(world)); + } + + @Test + public void shouldPutNullOnDelete() throws Exception { + store.put(hi, there); + store.delete(hi); + assertThat(inner.get(hi), nullValue()); + } + + @Test + public void shouldReturnOldValueOnDelete() throws Exception { + store.put(hi, there); + assertThat(store.delete(hi), equalTo(there)); + } + + @Test + public void shouldLogKeyNullOnDelete() throws Exception { + store.put(hi, there); + store.delete(hi); + assertThat(sent.get(hi), nullValue()); + } + + @Test + public void shouldWriteToInnerOnPutIfAbsentNoPreviousValue() throws Exception { + store.putIfAbsent(hi, there); + assertThat(inner.get(hi), equalTo(there)); + } + + @Test + public void shouldNotWriteToInnerOnPutIfAbsentWhenValueForKeyExists() throws Exception { + store.put(hi, there); + store.putIfAbsent(hi, world); + assertThat(inner.get(hi), equalTo(there)); + } + + @Test + public void shouldWriteToChangelogOnPutIfAbsentWhenNoPreviousValue() throws Exception { + store.putIfAbsent(hi, there); + assertThat((byte[]) sent.get(hi), equalTo(there)); + } + + @Test + public void shouldNotWriteToChangeLogOnPutIfAbsentWhenValueForKeyExists() throws Exception { + store.put(hi, there); + store.putIfAbsent(hi, world); + assertThat((byte[]) sent.get(hi), equalTo(there)); + } + + @Test + public void shouldReturnCurrentValueOnPutIfAbsent() throws Exception { + store.put(hi, there); + assertThat(store.putIfAbsent(hi, world), equalTo(there)); + } + + @Test + public void shouldReturnNullOnPutIfAbsentWhenNoPreviousValue() throws Exception { + assertThat(store.putIfAbsent(hi, there), is(nullValue())); + } + + @Test + public void shouldReturnValueOnGetWhenExists() throws Exception { + store.put(hello, world); + assertThat(store.get(hello), equalTo(world)); + } + + @Test + public void shouldReturnNullOnGetWhenDoesntExist() throws Exception { + assertThat(store.get(hello), is(nullValue())); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java new file mode 100644 index 0000000000000..8815c5a510e5f --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java @@ -0,0 +1,207 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.state.internals; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.test.InMemoryKeyValueStore; +import org.apache.kafka.test.MockProcessorContext; +import org.apache.kafka.test.NoOpRecordCollector; +import org.apache.kafka.test.TestUtils; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertFalse; + +public class ChangeLoggingKeyValueStoreTest { + + private final InMemoryKeyValueStore inner = new InMemoryKeyValueStore<>("kv"); + private final Serde keySerde = Serdes.String(); + private final Serde valueSerde = Serdes.String(); + private final ChangeLoggingKeyValueStore store + = new ChangeLoggingKeyValueStore<>(inner, keySerde, valueSerde); + private final Map sent = new HashMap<>(); + private final String hi = "hi"; + private final Bytes hiBytes = Bytes.wrap(hi.getBytes()); + private final String there = "there"; + private final byte[] thereBytes = "there".getBytes(); + private final String hello = "hello"; + private final String world = "world"; + + @Before + public void before() { + final NoOpRecordCollector collector = new NoOpRecordCollector() { + @Override + public void send(final ProducerRecord record, final Serializer keySerializer, final Serializer valueSerializer) { + sent.put(record.key(), record.value()); + } + }; + final MockProcessorContext context = new MockProcessorContext(null, + TestUtils.tempDirectory(), + Serdes.String(), + Serdes.Long(), + collector, + new ThreadCache("testCache", 0, new MockStreamsMetrics(new Metrics()))); + context.setTime(0); + store.init(context, store); + } + + @Test + public void shouldWriteKeyValueBytesToInnerStoreOnPut() throws Exception { + store.put(hi, there); + assertThat(deserializedValueFromInner(hi), equalTo(there)); + } + + @Test + public void shouldLogChangeOnPut() throws Exception { + store.put(hi, there); + assertThat((byte[]) sent.get(hiBytes), equalTo(thereBytes)); + } + + @Test + public void shouldWriteAllKeyValueToInnerStoreOnPutAll() throws Exception { + store.putAll(Arrays.asList(KeyValue.pair(hello, world), + KeyValue.pair(hi, there))); + assertThat(deserializedValueFromInner(hello), equalTo(world)); + assertThat(deserializedValueFromInner(hi), equalTo(there)); + } + + @Test + public void shouldLogChangesOnPutAll() throws Exception { + store.putAll(Arrays.asList(KeyValue.pair(hi, there), + KeyValue.pair(hello, world))); + assertThat((byte[]) sent.get(hiBytes), equalTo(thereBytes)); + assertThat((byte[]) sent.get(Bytes.wrap(hello.getBytes())), equalTo(world.getBytes())); + } + + @Test + public void shouldPutNullOnDelete() throws Exception { + store.put(hi, there); + store.delete(hi); + assertThat(inner.get(hiBytes), nullValue()); + } + + @Test + public void shouldReturnOldValueOnDelete() throws Exception { + store.put(hi, there); + assertThat(store.delete(hi), equalTo(there)); + } + + @Test + public void shouldReturnNullOnDeleteIfNoOldValue() throws Exception { + assertThat(store.delete(hi), is(nullValue())); + } + + @Test + public void shouldLogKeyNullOnDelete() throws Exception { + store.put(hi, there); + store.delete(hi); + assertThat(sent.get(hi), nullValue()); + } + + @Test + public void shouldWriteToInnerOnPutIfAbsentNoPreviousValue() throws Exception { + store.putIfAbsent(hi, there); + assertThat(inner.get(hiBytes), equalTo(thereBytes)); + } + + @Test + public void shouldNotWriteToInnerOnPutIfAbsentWhenValueForKeyExists() throws Exception { + store.put(hi, there); + store.putIfAbsent(hi, world); + assertThat(inner.get(hiBytes), equalTo(thereBytes)); + } + + @Test + public void shouldWriteToChangelogOnPutIfAbsentWhenNoPreviousValue() throws Exception { + store.putIfAbsent(hi, there); + assertThat((byte[]) sent.get(hiBytes), equalTo(thereBytes)); + } + + @Test + public void shouldNotWriteToChangeLogOnPutIfAbsentWhenValueForKeyExists() throws Exception { + store.put(hi, there); + store.putIfAbsent(hi, world); + assertThat((byte[]) sent.get(hiBytes), equalTo(thereBytes)); + } + + @Test + public void shouldReturnCurrentValueOnPutIfAbsent() throws Exception { + store.put(hi, there); + assertThat(store.putIfAbsent(hi, world), equalTo(there)); + } + + @Test + public void shouldReturnNullOnPutIfAbsentWhenNoPreviousValue() throws Exception { + assertThat(store.putIfAbsent(hi, there), is(nullValue())); + } + + @Test + public void shouldQueryRange() throws Exception { + store.put(hello, world); + store.put(hi, there); + store.put("zooom", "home"); + final KeyValueIterator range = store.range(hello, hi); + assertThat(range.next(), equalTo(KeyValue.pair(hello, world))); + assertThat(range.next(), equalTo(KeyValue.pair(hi, there))); + assertFalse(range.hasNext()); + } + + @Test + public void shouldReturnAllKeyValues() throws Exception { + store.put(hello, world); + store.put(hi, there); + final String zooom = "zooom"; + final String home = "home"; + store.put(zooom, home); + final KeyValueIterator all = store.all(); + assertThat(all.next(), equalTo(KeyValue.pair(hello, world))); + assertThat(all.next(), equalTo(KeyValue.pair(hi, there))); + assertThat(all.next(), equalTo(KeyValue.pair(zooom, home))); + assertFalse(all.hasNext()); + } + + @Test + public void shouldReturnValueOnGetWhenExists() throws Exception { + store.put(hello, world); + assertThat(store.get(hello), equalTo(world)); + } + + @Test + public void shouldReturnNullOnGetWhenDoesntExist() throws Exception { + assertThat(store.get(hello), is(nullValue())); + } + + private String deserializedValueFromInner(final String key) { + return valueSerde.deserializer().deserialize("blah", inner.get(Bytes.wrap(key.getBytes()))); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java index 50845e8bc1875..0ebdd5cac3c35 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.state.internals; +import org.apache.kafka.streams.KeyValue; import org.apache.kafka.test.InMemoryKeyValueStore; import org.junit.Before; import org.junit.Test; @@ -37,7 +38,7 @@ public void setUp() throws Exception { } @Test - public void shouldPeekNext() throws Exception { + public void shouldPeekNextKey() throws Exception { store.put("A", "A"); final DelegatingPeekingKeyValueIterator peekingIterator = new DelegatingPeekingKeyValueIterator<>(name, store.all()); assertEquals("A", peekingIterator.peekNextKey()); @@ -45,6 +46,15 @@ public void shouldPeekNext() throws Exception { assertTrue(peekingIterator.hasNext()); } + @Test + public void shouldPeekNext() throws Exception { + store.put("A", "A"); + final DelegatingPeekingKeyValueIterator peekingIterator = new DelegatingPeekingKeyValueIterator<>(name, store.all()); + assertEquals(KeyValue.pair("A", "A"), peekingIterator.peekNext()); + assertEquals(KeyValue.pair("A", "A"), peekingIterator.peekNext()); + assertTrue(peekingIterator.hasNext()); + } + @Test public void shouldPeekAndIterate() throws Exception { final String[] kvs = {"a", "b", "c", "d", "e", "f"}; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIteratorTest.java new file mode 100644 index 0000000000000..e7c2eb31edff4 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIteratorTest.java @@ -0,0 +1,113 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.state.internals; + +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.internals.SessionKeySerde; +import org.apache.kafka.streams.kstream.internals.SessionWindow; +import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.test.KeyValueIteratorStub; +import org.junit.Test; + +import java.util.Collections; +import java.util.Iterator; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class MergedSortedCacheSessionStoreIteratorTest { + + private final String storeKey = "a"; + private final String cacheKey = "b"; + + private final SessionWindow storeWindow = new SessionWindow(0, 1); + private final Iterator, byte[]>> storeKvs = Collections.singleton( + KeyValue.pair(new Windowed<>(Bytes.wrap(storeKey.getBytes()), storeWindow), storeKey.getBytes())).iterator(); + private final SessionWindow cacheWindow = new SessionWindow(10, 20); + private final Iterator> cacheKvs = Collections.singleton(KeyValue.pair( + SessionKeySerde.toBinary( + new Windowed<>(cacheKey, cacheWindow), Serdes.String().serializer()), new LRUCacheEntry(cacheKey.getBytes()))) + .iterator(); + + @Test + public void shouldHaveNextFromStore() throws Exception { + final MergedSortedCacheSessionStoreIterator mergeIterator + = createIterator(storeKvs, Collections.>emptyIterator()); + assertTrue(mergeIterator.hasNext()); + } + + @Test + public void shouldGetNextFromStore() throws Exception { + final MergedSortedCacheSessionStoreIterator mergeIterator + = createIterator(storeKvs, Collections.>emptyIterator()); + assertThat(mergeIterator.next(), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey))); + } + + @Test + public void shouldPeekNextKeyFromStore() throws Exception { + final MergedSortedCacheSessionStoreIterator mergeIterator + = createIterator(storeKvs, Collections.>emptyIterator()); + assertThat(mergeIterator.peekNextKey(), equalTo(new Windowed<>(storeKey, storeWindow))); + } + + @Test + public void shouldHaveNextFromCache() throws Exception { + final MergedSortedCacheSessionStoreIterator mergeIterator + = createIterator(Collections., byte[]>>emptyIterator(), + cacheKvs); + assertTrue(mergeIterator.hasNext()); + } + + @Test + public void shouldGetNextFromCache() throws Exception { + final MergedSortedCacheSessionStoreIterator mergeIterator + = createIterator(Collections., byte[]>>emptyIterator(), cacheKvs); + assertThat(mergeIterator.next(), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey))); + } + + @Test + public void shouldPeekNextKeyFromCache() throws Exception { + final MergedSortedCacheSessionStoreIterator mergeIterator + = createIterator(Collections., byte[]>>emptyIterator(), cacheKvs); + assertThat(mergeIterator.peekNextKey(), equalTo(new Windowed<>(cacheKey, cacheWindow))); + } + + @Test + public void shouldIterateBothStoreAndCache() throws Exception { + final MergedSortedCacheSessionStoreIterator iterator = createIterator(storeKvs, cacheKvs); + assertThat(iterator.next(), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey))); + assertThat(iterator.next(), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey))); + assertFalse(iterator.hasNext()); + } + + private MergedSortedCacheSessionStoreIterator createIterator(final Iterator, byte[]>> storeKvs, + final Iterator> cacheKvs) { + final DelegatingPeekingKeyValueIterator, byte[]> storeIterator + = new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(storeKvs)); + + final PeekingKeyValueIterator cacheIterator + = new DelegatingPeekingKeyValueIterator<>("cache", new KeyValueIteratorStub<>(cacheKvs)); + return new MergedSortedCacheSessionStoreIterator<>(cacheIterator, storeIterator, new StateSerdes<>("name", Serdes.String(), Serdes.String())); + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIteratorTest.java index b04f248e9ba89..376fca899342f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIteratorTest.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.state.KeyValueIterator; @@ -30,23 +29,25 @@ import java.util.ArrayList; import java.util.List; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; public class MergedSortedCacheWindowStoreIteratorTest { + private final List> windowStoreKvPairs = new ArrayList<>(); + private final ThreadCache cache = new ThreadCache("testCache", 1000000L, new MockStreamsMetrics(new Metrics())); + private final String namespace = "one"; + private final StateSerdes stateSerdes = new StateSerdes<>("foo", Serdes.String(), Serdes.String()); + @Test public void shouldIterateOverValueFromBothIterators() throws Exception { - final List> storeValues = new ArrayList<>(); - final ThreadCache cache = new ThreadCache("testCache", 1000000L, new MockStreamsMetrics(new Metrics())); - final String namespace = "one"; - final StateSerdes stateSerdes = new StateSerdes<>("foo", Serdes.String(), Serdes.String()); final List> expectedKvPairs = new ArrayList<>(); - for (long t = 0; t < 100; t += 20) { final byte[] v1Bytes = String.valueOf(t).getBytes(); - final KeyValue v1 = KeyValue.pair(Bytes.wrap(WindowStoreUtils.toBinaryKey("a", t, 0, stateSerdes)), v1Bytes); - storeValues.add(v1); + final KeyValue v1 = KeyValue.pair(t, v1Bytes); + windowStoreKvPairs.add(v1); expectedKvPairs.add(KeyValue.pair(t, v1Bytes)); final byte[] keyBytes = WindowStoreUtils.toBinaryKey("a", t + 10, 0, stateSerdes); final byte[] valBytes = String.valueOf(t + 10).getBytes(); @@ -56,11 +57,11 @@ public void shouldIterateOverValueFromBothIterators() throws Exception { byte[] binaryFrom = WindowStoreUtils.toBinaryKey("a", 0, 0, stateSerdes); byte[] binaryTo = WindowStoreUtils.toBinaryKey("a", 100, 0, stateSerdes); - final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>("name", new KeyValueIteratorStub<>(storeValues.iterator())); + final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>("name", new KeyValueIteratorStub<>(windowStoreKvPairs.iterator())); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(namespace, binaryFrom, binaryTo); - final MergedSortedCachedWindowStoreIterator iterator = new MergedSortedCachedWindowStoreIterator<>(cacheIterator, storeIterator, new StateSerdes<>("name", Serdes.Bytes(), Serdes.ByteArray())); + final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator<>(cacheIterator, storeIterator, new StateSerdes<>("name", Serdes.Long(), Serdes.ByteArray())); int index = 0; while (iterator.hasNext()) { final KeyValue next = iterator.next(); @@ -70,4 +71,18 @@ public void shouldIterateOverValueFromBothIterators() throws Exception { } } + @Test + public void shouldPeekNextKey() throws Exception { + windowStoreKvPairs.add(KeyValue.pair(10L, "a".getBytes())); + cache.put(namespace, WindowStoreUtils.toBinaryKey("a", 0, 0, stateSerdes), new LRUCacheEntry("b".getBytes())); + byte[] binaryFrom = WindowStoreUtils.toBinaryKey("a", 0, 0, stateSerdes); + byte[] binaryTo = WindowStoreUtils.toBinaryKey("a", 100, 0, stateSerdes); + final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>("name", new KeyValueIteratorStub<>(windowStoreKvPairs.iterator())); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(namespace, binaryFrom, binaryTo); + final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator<>(cacheIterator, storeIterator, new StateSerdes<>("name", Serdes.Long(), Serdes.ByteArray())); + assertThat(iterator.peekNextKey(), equalTo(0L)); + iterator.next(); + assertThat(iterator.peekNextKey(), equalTo(10L)); + } + } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java index 2082e00cba0bc..a2ce96c545dfb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java @@ -104,6 +104,11 @@ public void close() { } + @Override + public Long peekNextKey() { + throw new UnsupportedOperationException("peekNextKey not supported in stub"); + } + @Override public boolean hasNext() { return underlying.hasNext(); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplierTest.java new file mode 100644 index 0000000000000..3d9a56c271d6c --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplierTest.java @@ -0,0 +1,155 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.state.internals; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.test.MockProcessorContext; +import org.apache.kafka.test.NoOpRecordCollector; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class RocksDBKeyValueStoreSupplierTest { + + private static final String STORE_NAME = "name"; + private final ThreadCache cache = new ThreadCache("test", 1024, new MockStreamsMetrics(new Metrics())); + private final MockProcessorContext context = new MockProcessorContext(null, + TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + new NoOpRecordCollector(), + cache); + private KeyValueStore store; + + @After + public void close() { + store.close(); + } + + @Test + public void shouldCreateLoggingEnabledStoreWhenStoreLogged() throws Exception { + store = createStore(true, false); + final List logged = new ArrayList<>(); + final NoOpRecordCollector collector = new NoOpRecordCollector() { + @Override + public void send(final ProducerRecord record, final Serializer keySerializer, final Serializer valueSerializer) { + logged.add(record); + } + }; + final MockProcessorContext context = new MockProcessorContext(null, + TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + collector, + cache); + context.setTime(1); + store.init(context, store); + store.put("a", "b"); + assertFalse(logged.isEmpty()); + } + + @Test + public void shouldNotBeLoggingEnabledStoreWhenLoggingNotEnabled() throws Exception { + store = createStore(false, false); + final List logged = new ArrayList<>(); + final NoOpRecordCollector collector = new NoOpRecordCollector() { + @Override + public void send(final ProducerRecord record, final Serializer keySerializer, final Serializer valueSerializer) { + logged.add(record); + } + }; + final MockProcessorContext context = new MockProcessorContext(null, + TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + collector, + cache); + context.setTime(1); + store.init(context, store); + store.put("a", "b"); + assertTrue(logged.isEmpty()); + } + + @Test + public void shouldReturnCachedKeyValueStoreWhenCachingEnabled() throws Exception { + store = createStore(false, true); + store.init(context, store); + context.setTime(1); + store.put("a", "b"); + store.put("b", "c"); + assertThat(store, is(instanceOf(CachingKeyValueStore.class))); + assertThat(cache.size(), is(2L)); + } + + @Test + public void shouldReturnMeteredStoreWhenCachingAndLoggingDisabled() throws Exception { + store = createStore(false, false); + assertThat(store, is(instanceOf(MeteredKeyValueStore.class))); + } + + @Test + public void shouldReturnMeteredStoreWhenCachingDisabled() throws Exception { + store = createStore(true, false); + assertThat(store, is(instanceOf(MeteredKeyValueStore.class))); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldHaveMeteredStoreWhenCached() throws Exception { + store = createStore(false, true); + store.init(context, store); + final StreamsMetrics metrics = context.metrics(); + assertFalse(metrics.metrics().isEmpty()); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldHaveMeteredStoreWhenLogged() throws Exception { + store = createStore(true, false); + store.init(context, store); + final StreamsMetrics metrics = context.metrics(); + assertFalse(metrics.metrics().isEmpty()); + } + + @SuppressWarnings("unchecked") + private KeyValueStore createStore(final boolean logged, final boolean cached) { + return new RocksDBKeyValueStoreSupplier<>(STORE_NAME, + Serdes.String(), + Serdes.String(), + logged, + Collections.EMPTY_MAP, + cached).get(); + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplierTest.java new file mode 100644 index 0000000000000..28196a2a6ef8f --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplierTest.java @@ -0,0 +1,169 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.state.internals; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.internals.SessionWindow; +import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; +import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.test.MockProcessorContext; +import org.apache.kafka.test.NoOpRecordCollector; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class RocksDBSessionStoreSupplierTest { + + private static final String STORE_NAME = "name"; + private final ThreadCache cache = new ThreadCache("test", 1024, new MockStreamsMetrics(new Metrics())); + private final MockProcessorContext context = new MockProcessorContext(null, + TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + new NoOpRecordCollector(), + cache); + + private SessionStore store; + + @After + public void close() { + store.close(); + } + + @Test + public void shouldCreateLoggingEnabledStoreWhenStoreLogged() throws Exception { + store = createStore(true, false); + final List logged = new ArrayList<>(); + final NoOpRecordCollector collector = new NoOpRecordCollector() { + @Override + public void send(final ProducerRecord record, final Serializer keySerializer, final Serializer valueSerializer) { + logged.add(record); + } + }; + final MockProcessorContext context = new MockProcessorContext(null, + TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + collector, + cache); + context.setTime(1); + store.init(context, store); + store.put(new Windowed<>("a", new SessionWindow(0, 10)), "b"); + assertFalse(logged.isEmpty()); + } + + @Test + public void shouldNotBeLoggingEnabledStoreWhenLoggingNotEnabled() throws Exception { + store = createStore(false, false); + final List logged = new ArrayList<>(); + final NoOpRecordCollector collector = new NoOpRecordCollector() { + @Override + public void send(final ProducerRecord record, final Serializer keySerializer, final Serializer valueSerializer) { + logged.add(record); + } + }; + final MockProcessorContext context = new MockProcessorContext(null, + TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + collector, + cache); + context.setTime(1); + store.init(context, store); + store.put(new Windowed<>("a", new SessionWindow(0, 10)), "b"); + assertTrue(logged.isEmpty()); + } + + @Test + public void shouldReturnCachedSessionStoreWhenCachingEnabled() throws Exception { + store = createStore(false, true); + store.init(context, store); + context.setTime(1); + store.put(new Windowed<>("a", new SessionWindow(0, 10)), "b"); + store.put(new Windowed<>("b", new SessionWindow(0, 10)), "c"); + assertThat(store, is(instanceOf(CachingSessionStore.class))); + assertThat(cache.size(), is(2L)); + } + + @Test + public void shouldReturnRocksDbStoreWhenCachingAndLoggingDisabled() throws Exception { + store = createStore(false, false); + assertThat(store, is(instanceOf(RocksDBSessionStore.class))); + } + + @Test + public void shouldReturnRocksDbStoreWhenCachingDisabled() throws Exception { + store = createStore(true, false); + assertThat(store, is(instanceOf(RocksDBSessionStore.class))); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldHaveMeteredStoreWhenCached() throws Exception { + store = createStore(false, true); + store.init(context, store); + final StreamsMetrics metrics = context.metrics(); + assertFalse(metrics.metrics().isEmpty()); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldHaveMeteredStoreWhenLogged() throws Exception { + store = createStore(true, false); + store.init(context, store); + final StreamsMetrics metrics = context.metrics(); + assertFalse(metrics.metrics().isEmpty()); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldHaveMeteredStoreWhenNotLoggedOrCached() throws Exception { + store = createStore(false, false); + store.init(context, store); + final StreamsMetrics metrics = context.metrics(); + assertFalse(metrics.metrics().isEmpty()); + } + + + + private SessionStore createStore(final boolean logged, final boolean cached) { + return new RocksDBSessionStoreSupplier<>(STORE_NAME, + 10, + Serdes.String(), + Serdes.String(), + logged, + Collections.emptyMap(), + cached).get(); + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplierTest.java new file mode 100644 index 0000000000000..897ec623dc479 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplierTest.java @@ -0,0 +1,168 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.state.internals; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; +import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.test.MockProcessorContext; +import org.apache.kafka.test.NoOpRecordCollector; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class RocksDBWindowStoreSupplierTest { + + private static final String STORE_NAME = "name"; + private WindowStore store; + private final ThreadCache cache = new ThreadCache("test", 1024, new MockStreamsMetrics(new Metrics())); + private final MockProcessorContext context = new MockProcessorContext(null, + TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + new NoOpRecordCollector(), + cache); + + @After + public void close() { + store.close(); + } + + @Test + public void shouldCreateLoggingEnabledStoreWhenWindowStoreLogged() throws Exception { + store = createStore(true, false); + final List logged = new ArrayList<>(); + final NoOpRecordCollector collector = new NoOpRecordCollector() { + @Override + public void send(final ProducerRecord record, final Serializer keySerializer, final Serializer valueSerializer) { + logged.add(record); + } + }; + final MockProcessorContext context = new MockProcessorContext(null, + TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + collector, + cache); + context.setTime(1); + store.init(context, store); + store.put("a", "b"); + assertFalse(logged.isEmpty()); + } + + @Test + public void shouldNotBeLoggingEnabledStoreWhenLogginNotEnabled() throws Exception { + store = createStore(false, false); + final List logged = new ArrayList<>(); + final NoOpRecordCollector collector = new NoOpRecordCollector() { + @Override + public void send(final ProducerRecord record, final Serializer keySerializer, final Serializer valueSerializer) { + logged.add(record); + } + }; + final MockProcessorContext context = new MockProcessorContext(null, + TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + collector, + cache); + context.setTime(1); + store.init(context, store); + store.put("a", "b"); + assertTrue(logged.isEmpty()); + } + + @Test + public void shouldBeCachedWindowStoreWhenCachingEnabled() throws Exception { + store = createStore(false, true); + store.init(context, store); + context.setTime(1); + store.put("a", "b"); + store.put("b", "c"); + assertThat(store, is(instanceOf(CachingWindowStore.class))); + assertThat(context.getCache().size(), is(2L)); + } + + @Test + public void shouldReturnRocksDbStoreWhenCachingAndLoggingDisabled() throws Exception { + store = createStore(false, false); + assertThat(store, is(instanceOf(RocksDBWindowStore.class))); + } + + @Test + public void shouldReturnRocksDbStoreWhenCachingDisabled() throws Exception { + store = createStore(true, false); + assertThat(store, is(instanceOf(RocksDBWindowStore.class))); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldHaveMeteredStoreWhenCached() throws Exception { + store = createStore(false, true); + store.init(context, store); + final StreamsMetrics metrics = context.metrics(); + assertFalse(metrics.metrics().isEmpty()); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldHaveMeteredStoreWhenLogged() throws Exception { + store = createStore(true, false); + store.init(context, store); + final StreamsMetrics metrics = context.metrics(); + assertFalse(metrics.metrics().isEmpty()); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldHaveMeteredStoreWhenNotLoggedOrCached() throws Exception { + store = createStore(false, false); + store.init(context, store); + final StreamsMetrics metrics = context.metrics(); + assertFalse(metrics.metrics().isEmpty()); + } + + @SuppressWarnings("unchecked") + private WindowStore createStore(final boolean logged, final boolean cached) { + return new RocksDBWindowStoreSupplier<>(STORE_NAME, + 10, + 3, + false, + Serdes.String(), + Serdes.String(), + 10, + logged, + Collections.emptyMap(), + cached).get(); + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java index a5225926e27ed..79223dee81ab3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java @@ -490,8 +490,6 @@ public void send(ProducerRecord record, Serializer keySeria recordCollector, cache); WindowStore store = createWindowStore(context, false, true); - RocksDBWindowStore inner = - (RocksDBWindowStore) ((MeteredWindowStore) store).inner(); try { // to validate segments final Segments segments = new Segments(windowName, retentionPeriod, numSegments); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIteratorTest.java new file mode 100644 index 0000000000000..8c0d2fec0f5db --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIteratorTest.java @@ -0,0 +1,95 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.state.internals; + +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.test.KeyValueIteratorStub; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Iterator; +import java.util.NoSuchElementException; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class SerializedKeyValueIteratorTest { + + private final StateSerdes serdes = new StateSerdes<>("blah", Serdes.String(), Serdes.String()); + private final Iterator> iterator + = Arrays.asList(KeyValue.pair(Bytes.wrap("hi".getBytes()), "there".getBytes()), + KeyValue.pair(Bytes.wrap("hello".getBytes()), "world".getBytes())) + .iterator(); + private final DelegatingPeekingKeyValueIterator peeking + = new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(iterator)); + private final SerializedKeyValueIterator serializedKeyValueIterator + = new SerializedKeyValueIterator<>(peeking, serdes); + + @Test + public void shouldReturnTrueOnHasNextWhenMoreResults() { + assertTrue(serializedKeyValueIterator.hasNext()); + } + + @Test + public void shouldReturnNextValueWhenItExists() throws Exception { + assertThat(serializedKeyValueIterator.next(), equalTo(KeyValue.pair("hi", "there"))); + assertThat(serializedKeyValueIterator.next(), equalTo(KeyValue.pair("hello", "world"))); + } + + @Test + public void shouldReturnFalseOnHasNextWhenNoMoreResults() throws Exception { + advanceIteratorToEnd(); + assertFalse(serializedKeyValueIterator.hasNext()); + } + + @Test + public void shouldThrowNoSuchElementOnNextWhenIteratorExhausted() throws Exception { + advanceIteratorToEnd(); + try { + serializedKeyValueIterator.next(); + fail("Expected NoSuchElementException on exhausted iterator"); + } catch (final NoSuchElementException nse) { + // pass + } + } + + @Test + public void shouldPeekNextKey() throws Exception { + assertThat(serializedKeyValueIterator.peekNextKey(), equalTo("hi")); + serializedKeyValueIterator.next(); + assertThat(serializedKeyValueIterator.peekNextKey(), equalTo("hello")); + } + + @Test(expected = UnsupportedOperationException.class) + public void shouldThrowUnsupportedOperationOnRemove() throws Exception { + serializedKeyValueIterator.remove(); + } + + private void advanceIteratorToEnd() { + serializedKeyValueIterator.next(); + serializedKeyValueIterator.next(); + } + + +} \ No newline at end of file From eb62e5695506ae13bd37102c3c08e8a067eca0c8 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 18 Jan 2017 02:43:10 +0000 Subject: [PATCH 008/177] KAFKA-4591; Create Topic Policy follow-up 1. Added javadoc to public classes 2. Removed `s` from config name for consistency with interface name 3. The policy interface now implements Configurable and AutoCloseable as per the KIP 4. Use `null` instead of `-1` in `RequestMetadata` 5. Perform all broker validation before invoking the policy 6. Add tests Author: Ismael Juma Reviewers: Jason Gustafson Closes #2388 from ijuma/create-topic-policy-docs-and-config-name-change (cherry picked from commit fd6d7bcf335166a524dc9a29a50c96af8f1c1c02) Signed-off-by: Ismael Juma --- .../errors/PolicyViolationException.java | 3 + .../apache/kafka/common/protocol/Errors.java | 2 +- .../server/policy/CreateTopicPolicy.java | 72 +++++++++++++++++-- .../main/scala/kafka/admin/AdminUtils.scala | 36 ++++++---- .../scala/kafka/server/AdminManager.scala | 56 ++++++++++----- .../main/scala/kafka/server/KafkaConfig.scala | 6 +- .../AbstractCreateTopicsRequestTest.scala | 2 +- .../CreateTopicsRequestWithPolicyTest.scala | 59 ++++++++++++--- .../unit/kafka/server/KafkaConfigTest.scala | 2 +- 9 files changed, 183 insertions(+), 55 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/errors/PolicyViolationException.java b/clients/src/main/java/org/apache/kafka/common/errors/PolicyViolationException.java index 7923444c2a8f6..393a6df4bf6dd 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/PolicyViolationException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/PolicyViolationException.java @@ -17,6 +17,9 @@ package org.apache.kafka.common.errors; +/** + * Exception thrown if a create topics request does not satisfy the configured policy for a topic. + */ public class PolicyViolationException extends ApiException { public PolicyViolationException(String message) { diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index f30f889054cec..e7689e2e11908 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -166,7 +166,7 @@ public enum Errors { " the message was sent to an incompatible broker. See the broker logs for more details.")), UNSUPPORTED_FOR_MESSAGE_FORMAT(43, new UnsupportedForMessageFormatException("The message format version on the broker does not support the request.")), - POLICY_VIOLATION(44, new PolicyViolationException("Request parameters do not satisfy the system policy.")); + POLICY_VIOLATION(44, new PolicyViolationException("Request parameters do not satisfy the configured policy.")); private static final Logger log = LoggerFactory.getLogger(Errors.class); diff --git a/clients/src/main/java/org/apache/kafka/server/policy/CreateTopicPolicy.java b/clients/src/main/java/org/apache/kafka/server/policy/CreateTopicPolicy.java index 94f1e763a222c..22a7c1d37ccb4 100644 --- a/clients/src/main/java/org/apache/kafka/server/policy/CreateTopicPolicy.java +++ b/clients/src/main/java/org/apache/kafka/server/policy/CreateTopicPolicy.java @@ -13,42 +13,90 @@ package org.apache.kafka.server.policy; +import org.apache.kafka.common.Configurable; import org.apache.kafka.common.errors.PolicyViolationException; import java.util.Collections; import java.util.List; import java.util.Map; -public interface CreateTopicPolicy { +/** + * An interface for enforcing a policy on create topics requests. + * + * Common use cases are requiring that the replication factor, min.insync.replicas and/or retention settings for a + * topic are within an allowable range. + * + * If create.topic.policy.class.name is defined, Kafka will create an instance of the specified class + * using the default constructor and will then pass the broker configs to its configure() method. During + * broker shutdown, the close() method will be invoked so that resources can be released (if necessary). + */ +public interface CreateTopicPolicy extends Configurable, AutoCloseable { + /** + * Class containing the create request parameters. + */ class RequestMetadata { private final String topic; - private final int numPartitions; - private final short replicationFactor; + private final Integer numPartitions; + private final Short replicationFactor; private final Map> replicasAssignments; private final Map configs; - public RequestMetadata(String topic, int numPartitions, short replicationFactor, - Map> replicasAssignments, Map configs) { + /** + * Create an instance of this class with the provided parameters. + * + * This constructor is public to make testing of CreateTopicPolicy implementations easier. + * + * @param topic the name of the topic to created. + * @param numPartitions the number of partitions to create or null if replicasAssignments is set. + * @param replicationFactor the replication factor for the topic or null if replicaAssignments is set. + * @param replicasAssignments replica assignments or null if numPartitions and replicationFactor is set. The + * assignment is a map from partition id to replica (broker) ids. + * @param configs topic configs for the topic to be created, not including broker defaults. Broker configs are + * passed via the {@code configure()} method of the policy implementation. + */ + public RequestMetadata(String topic, Integer numPartitions, Short replicationFactor, + Map> replicasAssignments, Map configs) { this.topic = topic; this.numPartitions = numPartitions; this.replicationFactor = replicationFactor; - this.replicasAssignments = Collections.unmodifiableMap(replicasAssignments); + this.replicasAssignments = replicasAssignments == null ? null : Collections.unmodifiableMap(replicasAssignments); this.configs = Collections.unmodifiableMap(configs); } + /** + * Return the name of the topic to create. + */ public String topic() { return topic; } - public int numPartitions() { + /** + * Return the number of partitions to create or null if replicaAssignments is not null. + */ + public Integer numPartitions() { return numPartitions; } + /** + * Return the number of replicas to create or null if replicaAssignments is not null. + */ + public Short replicationFactor() { + return replicationFactor; + } + + /** + * Return a map from partition id to replica (broker) ids or null if numPartitions and replicationFactor are + * set instead. + */ public Map> replicasAssignments() { return replicasAssignments; } + /** + * Return topic configs in the request, not including broker defaults. Broker configs are passed via + * the {@code configure()} method of the policy implementation. + */ public Map configs() { return configs; } @@ -63,5 +111,15 @@ public String toString() { } } + /** + * Validate the request parameters and throw a PolicyViolationException with a suitable error + * message if the create request parameters for the provided topic do not satisfy this policy. + * + * Clients will receive the POLICY_VIOLATION error code along with the exception's message. Note that validation + * failure only affects the relevant topic, other topics in the request will still be processed. + * + * @param requestMetadata the create request parameters for the provided topic. + * @throws PolicyViolationException if the request parameters do not satisfy this policy. + */ void validate(RequestMetadata requestMetadata) throws PolicyViolationException; } diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 916db483338de..02d5fe07961d0 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -417,12 +417,11 @@ object AdminUtils extends Logging with AdminUtilities { AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, replicaAssignment, topicConfig) } - def createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils: ZkUtils, - topic: String, - partitionReplicaAssignment: Map[Int, Seq[Int]], - config: Properties = new Properties, - update: Boolean = false, - validateOnly: Boolean = false) { + def validateCreateOrUpdateTopic(zkUtils: ZkUtils, + topic: String, + partitionReplicaAssignment: Map[Int, Seq[Int]], + config: Properties, + update: Boolean): Unit = { // validate arguments Topic.validate(topic) @@ -450,18 +449,25 @@ object AdminUtils extends Logging with AdminUtilities { // Configs only matter if a topic is being created. Changing configs via AlterTopic is not supported - if (!update) { + if (!update) LogConfig.validate(config) - if (!validateOnly) { - // write out the config if there is any, this isn't transactional with the partition assignments - writeEntityConfig(zkUtils, getEntityConfigPath(ConfigType.Topic, topic), config) - } - } + } - if (!validateOnly) { - // create the partition assignment - writeTopicPartitionAssignment(zkUtils, topic, partitionReplicaAssignment, update) + def createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils: ZkUtils, + topic: String, + partitionReplicaAssignment: Map[Int, Seq[Int]], + config: Properties = new Properties, + update: Boolean = false) { + validateCreateOrUpdateTopic(zkUtils, topic, partitionReplicaAssignment, config, update) + + // Configs only matter if a topic is being created. Changing configs via AlterTopic is not supported + if (!update) { + // write out the config if there is any, this isn't transactional with the partition assignments + writeEntityConfig(zkUtils, getEntityConfigPath(ConfigType.Topic, topic), config) } + + // create the partition assignment + writeTopicPartitionAssignment(zkUtils, topic, partitionReplicaAssignment, update) } private def writeTopicPartitionAssignment(zkUtils: ZkUtils, topic: String, replicaAssignment: Map[Int, Seq[Int]], update: Boolean) { diff --git a/core/src/main/scala/kafka/server/AdminManager.scala b/core/src/main/scala/kafka/server/AdminManager.scala index 75a71f5124083..6dc224dc54603 100644 --- a/core/src/main/scala/kafka/server/AdminManager.scala +++ b/core/src/main/scala/kafka/server/AdminManager.scala @@ -23,7 +23,7 @@ import kafka.common.TopicAlreadyMarkedForDeletionException import kafka.log.LogConfig import kafka.metrics.KafkaMetricsGroup import kafka.utils._ -import org.apache.kafka.common.errors.InvalidRequestException +import org.apache.kafka.common.errors.{ApiException, InvalidRequestException, PolicyViolationException} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.CreateTopicsRequest._ @@ -43,7 +43,7 @@ class AdminManager(val config: KafkaConfig, private val topicPurgatory = DelayedOperationPurgatory[DelayedOperation]("topic", config.brokerId) private val createTopicPolicy = - Option(config.getConfiguredInstance(KafkaConfig.CreateTopicsPolicyClassNameProp, classOf[CreateTopicPolicy])) + Option(config.getConfiguredInstance(KafkaConfig.CreateTopicPolicyClassNameProp, classOf[CreateTopicPolicy])) def hasDelayedTopicOperations = topicPurgatory.delayed() != 0 @@ -80,29 +80,49 @@ class AdminManager(val config: KafkaConfig, && !arguments.replicasAssignments.isEmpty) throw new InvalidRequestException("Both numPartitions or replicationFactor and replicasAssignments were set. " + "Both cannot be used at the same time.") - else { - createTopicPolicy.foreach(_.validate(new RequestMetadata(topic, arguments.numPartitions, - arguments.replicationFactor, arguments.replicasAssignments, arguments.configs))) - - if (!arguments.replicasAssignments.isEmpty) { - // Note: we don't check that replicaAssignment doesn't contain unknown brokers - unlike in add-partitions case, - // this follows the existing logic in TopicCommand - arguments.replicasAssignments.asScala.map { case (partitionId, replicas) => - (partitionId.intValue, replicas.asScala.map(_.intValue)) - } - } else { - AdminUtils.assignReplicasToBrokers(brokers, arguments.numPartitions, arguments.replicationFactor) + else if (!arguments.replicasAssignments.isEmpty) { + // Note: we don't check that replicaAssignment contains unknown brokers - unlike in add-partitions case, + // this follows the existing logic in TopicCommand + arguments.replicasAssignments.asScala.map { case (partitionId, replicas) => + (partitionId.intValue, replicas.asScala.map(_.intValue)) } - } + } else + AdminUtils.assignReplicasToBrokers(brokers, arguments.numPartitions, arguments.replicationFactor) } trace(s"Assignments for topic $topic are $assignments ") - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, assignments, configs, - update = false, validateOnly = validateOnly) + + createTopicPolicy match { + case Some(policy) => + AdminUtils.validateCreateOrUpdateTopic(zkUtils, topic, assignments, configs, update = false) + + // Use `null` for unset fields in the public API + val numPartitions: java.lang.Integer = + if (arguments.numPartitions == NO_NUM_PARTITIONS) null else arguments.numPartitions + val replicationFactor: java.lang.Short = + if (arguments.replicationFactor == NO_REPLICATION_FACTOR) null else arguments.replicationFactor + val replicaAssignments = if (arguments.replicasAssignments.isEmpty) null else arguments.replicasAssignments + + policy.validate(new RequestMetadata(topic, numPartitions, replicationFactor, replicaAssignments, + arguments.configs)) + + if (!validateOnly) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, assignments, configs, update = false) + + case None => + if (validateOnly) + AdminUtils.validateCreateOrUpdateTopic(zkUtils, topic, assignments, configs, update = false) + else + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, assignments, configs, update = false) + } CreateTopicMetadata(topic, assignments, new CreateTopicsResponse.Error(Errors.NONE, null)) } catch { - case e: Throwable => + // Log client errors at a lower level than unexpected exceptions + case e@ (_: PolicyViolationException | _: ApiException) => info(s"Error processing create topic request for topic $topic with arguments $arguments", e) CreateTopicMetadata(topic, Map(), new CreateTopicsResponse.Error(Errors.forException(e), e.getMessage)) + case e: Throwable => + error(s"Error processing create topic request for topic $topic with arguments $arguments", e) + CreateTopicMetadata(topic, Map(), new CreateTopicsResponse.Error(Errors.forException(e), e.getMessage)) } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 891327f605d21..3c2a72d8ac6d9 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -281,7 +281,7 @@ object KafkaConfig { val NumRecoveryThreadsPerDataDirProp = "num.recovery.threads.per.data.dir" val AutoCreateTopicsEnableProp = "auto.create.topics.enable" val MinInSyncReplicasProp = "min.insync.replicas" - val CreateTopicsPolicyClassNameProp = "create.topics.policy.class.name" + val CreateTopicPolicyClassNameProp = "create.topic.policy.class.name" /** ********* Replication configuration ***********/ val ControllerSocketTimeoutMsProp = "controller.socket.timeout.ms" val DefaultReplicationFactorProp = "default.replication.factor" @@ -491,7 +491,7 @@ object KafkaConfig { "produce with acks of \"all\". This will ensure that the producer raises an exception " + "if a majority of replicas do not receive a write." - val CreateTopicsPolicyClassNameDoc = "The create topics policy class that should be used for validation. The class should " + + val CreateTopicPolicyClassNameDoc = "The create topic policy class that should be used for validation. The class should " + "implement the org.apache.kafka.server.policy.CreateTopicPolicy interface." /** ********* Replication configuration ***********/ val ControllerSocketTimeoutMsDoc = "The socket timeout for controller-to-broker channels" @@ -693,7 +693,7 @@ object KafkaConfig { .define(LogMessageFormatVersionProp, STRING, Defaults.LogMessageFormatVersion, MEDIUM, LogMessageFormatVersionDoc) .define(LogMessageTimestampTypeProp, STRING, Defaults.LogMessageTimestampType, in("CreateTime", "LogAppendTime"), MEDIUM, LogMessageTimestampTypeDoc) .define(LogMessageTimestampDifferenceMaxMsProp, LONG, Defaults.LogMessageTimestampDifferenceMaxMs, atLeast(0), MEDIUM, LogMessageTimestampDifferenceMaxMsDoc) - .define(CreateTopicsPolicyClassNameProp, CLASS, null, LOW, CreateTopicsPolicyClassNameDoc) + .define(CreateTopicPolicyClassNameProp, CLASS, null, LOW, CreateTopicPolicyClassNameDoc) /** ********* Replication configuration ***********/ .define(ControllerSocketTimeoutMsProp, INT, Defaults.ControllerSocketTimeoutMs, MEDIUM, ControllerSocketTimeoutMsDoc) diff --git a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala index 76774bdf55be3..32e69c772b39d 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala @@ -130,7 +130,7 @@ class AbstractCreateTopicsRequestTest extends BaseRequestTest { } protected def replicaAssignmentToJava(assignments: Map[Int, List[Int]]) = { - assignments.map { case (k, v) => (k:Integer, v.map { i => i:Integer }.asJava) }.asJava + assignments.map { case (k, v) => (k: Integer, v.map { i => i: Integer }.asJava) }.asJava } protected def sendCreateTopicRequest(request: CreateTopicsRequest, socketServer: SocketServer = controllerSocketServer): CreateTopicsResponse = { diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala index 7127eaffca9de..80f6e9e096816 100644 --- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala +++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala @@ -17,6 +17,7 @@ package kafka.server +import java.util import java.util.Properties import kafka.utils.TestUtils @@ -34,16 +35,22 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest override def propertyOverrides(properties: Properties): Unit = { super.propertyOverrides(properties) - properties.put(KafkaConfig.CreateTopicsPolicyClassNameProp, classOf[Policy].getName) + properties.put(KafkaConfig.CreateTopicPolicyClassNameProp, classOf[Policy].getName) } @Test def testValidCreateTopicsRequests() { val timeout = 10000 + validateValidCreateTopicsRequests(new CreateTopicsRequest.Builder( Map("topic1" -> new CreateTopicsRequest.TopicDetails(5, 1.toShort)).asJava, timeout).build()) + validateValidCreateTopicsRequests(new CreateTopicsRequest.Builder( Map("topic2" -> new CreateTopicsRequest.TopicDetails(5, 3.toShort)).asJava, timeout, true).build()) + + val assignments = replicaAssignmentToJava(Map(0 -> List(1, 0), 1 -> List(0, 1))) + validateValidCreateTopicsRequests(new CreateTopicsRequest.Builder( + Map("topic3" -> new CreateTopicsRequest.TopicDetails(assignments)).asJava, timeout).build()) } @Test @@ -54,28 +61,62 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest // Policy violations validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder( - Map("topic3" -> new CreateTopicsRequest.TopicDetails(4, 1.toShort)).asJava, timeout).build(), - Map("topic3" -> error(Errors.POLICY_VIOLATION, Some("Topics should have at least 5 partitions, received 4")))) + Map("policy-topic1" -> new CreateTopicsRequest.TopicDetails(4, 1.toShort)).asJava, timeout).build(), + Map("policy-topic1" -> error(Errors.POLICY_VIOLATION, Some("Topics should have at least 5 partitions, received 4")))) + + validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder( + Map("policy-topic2" -> new CreateTopicsRequest.TopicDetails(4, 1.toShort)).asJava, timeout, true).build(), + Map("policy-topic2" -> error(Errors.POLICY_VIOLATION, Some("Topics should have at least 5 partitions, received 4")))) + val assignments = replicaAssignmentToJava(Map(0 -> List(1), 1 -> List(0))) validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder( - Map("topic4" -> new CreateTopicsRequest.TopicDetails(4, 1.toShort)).asJava, timeout, true).build(), - Map("topic4" -> error(Errors.POLICY_VIOLATION, Some("Topics should have at least 5 partitions, received 4")))) + Map("policy-topic3" -> new CreateTopicsRequest.TopicDetails(assignments)).asJava, timeout).build(), + Map("policy-topic3" -> error(Errors.POLICY_VIOLATION, + Some("""Topic partitions should have at least 2 partitions, received 1 for partition 0"""))), checkErrorMessage = true) // Check that basic errors still work validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder( Map(existingTopic -> new CreateTopicsRequest.TopicDetails(5, 1.toShort)).asJava, timeout).build(), Map(existingTopic -> error(Errors.TOPIC_ALREADY_EXISTS, Some("""Topic "existing-topic" already exists.""")))) + validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder( Map("error-replication" -> new CreateTopicsRequest.TopicDetails(10, (numBrokers + 1).toShort)).asJava, timeout, true).build(), - Map("error-replication" -> error(Errors.INVALID_REPLICATION_FACTOR, Some("replication factor: 4 larger than available brokers: 3")))) + Map("error-replication" -> error(Errors.INVALID_REPLICATION_FACTOR, + Some("replication factor: 4 larger than available brokers: 3")))) + + validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder( + Map("error-replication2" -> new CreateTopicsRequest.TopicDetails(10, -1: Short)).asJava, timeout, true).build(), + Map("error-replication2" -> error(Errors.INVALID_REPLICATION_FACTOR, Some("replication factor must be larger than 0")))) } } object CreateTopicsRequestWithPolicyTest { + class Policy extends CreateTopicPolicy { - def validate(requestMetadata: RequestMetadata): Unit = - if (requestMetadata.numPartitions < 5) - throw new PolicyViolationException(s"Topics should have at least 5 partitions, received ${requestMetadata.numPartitions}") + def configure(configs: util.Map[String, _]): Unit = () + + def validate(requestMetadata: RequestMetadata): Unit = { + import requestMetadata._ + require(configs.isEmpty, s"Topic configs should be empty, but it is $configs") + if (numPartitions != null || replicationFactor != null) { + require(numPartitions != null, s"numPartitions should not be null, but it is $numPartitions") + require(replicationFactor != null, s"replicationFactor should not be null, but it is $replicationFactor") + require(replicasAssignments == null, s"replicaAssigments should be null, but it is $replicasAssignments") + if (numPartitions < 5) + throw new PolicyViolationException(s"Topics should have at least 5 partitions, received $numPartitions") + } else { + require(numPartitions == null, s"numPartitions should be null, but it is $numPartitions") + require(replicationFactor == null, s"replicationFactor should be null, but it is $replicationFactor") + require(replicasAssignments != null, s"replicaAssigments should not be null, but it is $replicasAssignments") + replicasAssignments.asScala.foreach { case (partitionId, assignment) => + if (assignment.size < 2) + throw new PolicyViolationException("Topic partitions should have at least 2 partitions, received " + + s"${assignment.size} for partition $partitionId") + } + } + } + + def close(): Unit = () } } diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index a5ca5a832e226..259178c57710b 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -547,7 +547,7 @@ class KafkaConfigTest { case KafkaConfig.RequestTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.AuthorizerClassNameProp => //ignore string - case KafkaConfig.CreateTopicsPolicyClassNameProp => //ignore string + case KafkaConfig.CreateTopicPolicyClassNameProp => //ignore string case KafkaConfig.PortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.HostNameProp => // ignore string From e38794e020951adec5a5d0bbfe42c57294bf67bd Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 17 Jan 2017 20:29:55 -0800 Subject: [PATCH 009/177] KAFKA-3502; move RocksDB options construction to init() In RocksDBStore, options / wOptions / fOptions are constructed in the constructor, which needs to be dismissed in the close() call; however in some tests, the generated topology is not initialized at all, and hence the corresponding state stores are supposed to not be able to be closed as well since their `init` function is not called. This could cause the above option objects to be not released. This is fixed in this patch to move the logic out of constructor and inside `init` functions, so that no RocksDB objects will be created in the constructor only. Also some minor cleanups: 1. In KStreamTestDriver.close(), we lost the logic to close the state stores but only call `flush`; it is now changed back to call both. 2. Moved the forwarding logic from KStreamTestDriver to MockProcessorContext to remove the mutual dependency: these functions should really be in ProcessorContext, not the test driver. Author: Guozhang Wang Reviewers: Damian Guy , Matthias J. Sax , Jason Gustafson Closes #2381 from guozhangwang/K3502-pure-virtual-function-unit-tests (cherry picked from commit 1974e1b0e54abe5fdebd8ff3338df864b7ab60f3) Signed-off-by: Jason Gustafson --- .../streams/state/internals/RocksDBStore.java | 21 +--- .../streams/kstream/KStreamBuilderTest.java | 58 ++++----- .../internals/KStreamKTableLeftJoinTest.java | 2 - ...amSessionWindowAggregateProcessorTest.java | 4 +- .../state/KeyValueStoreTestDriver.java | 2 +- .../internals/CachingKeyValueStoreTest.java | 2 +- .../internals/CachingSessionStoreTest.java | 2 +- .../internals/CachingWindowStoreTest.java | 2 +- .../ChangeLoggingKeyValueBytesStoreTest.java | 3 +- .../ChangeLoggingKeyValueStoreTest.java | 3 +- .../ChangeLoggingSegmentedBytesStoreTest.java | 3 +- .../MeteredSegmentedBytesStoreTest.java | 3 +- .../RocksDBKeyValueStoreSupplierTest.java | 9 +- .../RocksDBSegmentedBytesStoreTest.java | 3 +- .../RocksDBSessionStoreSupplierTest.java | 9 +- .../internals/RocksDBSessionStoreTest.java | 3 +- .../RocksDBWindowStoreSupplierTest.java | 9 +- .../internals/RocksDBWindowStoreTest.java | 24 ++-- .../state/internals/SegmentIteratorTest.java | 27 +++-- .../streams/state/internals/SegmentsTest.java | 3 +- .../apache/kafka/test/KStreamTestDriver.java | 113 +++++------------- .../kafka/test/MockProcessorContext.java | 57 ++++++--- 22 files changed, 153 insertions(+), 209 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index 3f8d509dda05a..55c1bb821c428 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -94,25 +94,21 @@ public class RocksDBStore implements KeyValueStore { protected volatile boolean open = false; - - public RocksDBStore(final String name, - final Serde keySerde, - final Serde valueSerde) { + RocksDBStore(String name, Serde keySerde, Serde valueSerde) { this(name, DB_FILE_DIR, keySerde, valueSerde); } - - public RocksDBStore(final String name, - final String parentDir, - final Serde keySerde, - final Serde valueSerde) { + RocksDBStore(String name, String parentDir, Serde keySerde, Serde valueSerde) { this.name = name; this.parentDir = parentDir; this.keySerde = keySerde; this.valueSerde = valueSerde; + } + @SuppressWarnings("unchecked") + public void openDB(ProcessorContext context) { // initialize the default rocksdb options - BlockBasedTableConfig tableConfig = new BlockBasedTableConfig(); + final BlockBasedTableConfig tableConfig = new BlockBasedTableConfig(); tableConfig.setBlockCacheSize(BLOCK_CACHE_SIZE); tableConfig.setBlockSize(BLOCK_SIZE); @@ -125,16 +121,12 @@ public RocksDBStore(final String name, options.setCreateIfMissing(true); options.setErrorIfExists(false); - wOptions = new WriteOptions(); wOptions.setDisableWAL(true); fOptions = new FlushOptions(); fOptions.setWaitForFlush(true); - } - @SuppressWarnings("unchecked") - public void openDB(ProcessorContext context) { final Map configs = context.appConfigs(); final Class configSetterClass = (Class) configs.get(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG); if (configSetterClass != null) { @@ -464,5 +456,4 @@ public synchronized boolean hasNext() { return super.hasNext() && comparator.compare(super.peekRawKey(), this.rawToKey) <= 0; } } - } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java index c32082c641c96..5f126c33e9d4f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.util.HashSet; @@ -41,8 +42,17 @@ public class KStreamBuilderTest { + private static final String APP_ID = "app-id"; + + private final KStreamBuilder builder = new KStreamBuilder(); + private KStreamTestDriver driver = null; + @Before + public void setUp() { + builder.setApplicationId(APP_ID); + } + @After public void cleanup() { if (driver != null) { @@ -53,8 +63,6 @@ public void cleanup() { @Test(expected = TopologyBuilderException.class) public void testFrom() { - final KStreamBuilder builder = new KStreamBuilder(); - builder.stream("topic-1", "topic-2"); builder.addSource(KStreamImpl.SOURCE_NAME + "0000000000", "topic-3"); @@ -62,17 +70,15 @@ public void testFrom() { @Test public void testNewName() { - KStreamBuilder builder = new KStreamBuilder(); - assertEquals("X-0000000000", builder.newName("X-")); assertEquals("Y-0000000001", builder.newName("Y-")); assertEquals("Z-0000000002", builder.newName("Z-")); - builder = new KStreamBuilder(); + KStreamBuilder newBuilder = new KStreamBuilder(); - assertEquals("X-0000000000", builder.newName("X-")); - assertEquals("Y-0000000001", builder.newName("Y-")); - assertEquals("Z-0000000002", builder.newName("Z-")); + assertEquals("X-0000000000", newBuilder.newName("X-")); + assertEquals("Y-0000000001", newBuilder.newName("Y-")); + assertEquals("Z-0000000002", newBuilder.newName("Z-")); } @Test @@ -80,8 +86,6 @@ public void testMerge() { String topic1 = "topic-1"; String topic2 = "topic-2"; - KStreamBuilder builder = new KStreamBuilder(); - KStream source1 = builder.stream(topic1); KStream source2 = builder.stream(topic2); KStream merged = builder.merge(source1, source2); @@ -105,7 +109,6 @@ public void shouldHaveCorrectSourceTopicsForTableFromMergedStream() throws Excep final String topic1 = "topic-1"; final String topic2 = "topic-2"; final String topic3 = "topic-3"; - final KStreamBuilder builder = new KStreamBuilder(); final KStream source1 = builder.stream(topic1); final KStream source2 = builder.stream(topic2); final KStream source3 = builder.stream(topic3); @@ -131,28 +134,26 @@ public boolean test(final String key, final String value) { final KStream merged = builder.merge(processedSource1, processedSource2, source3); merged.groupByKey().count("my-table"); final Map> actual = builder.stateStoreNameToSourceTopics(); + assertEquals(Utils.mkSet("topic-1", "topic-2", "topic-3"), actual.get("my-table")); } @Test(expected = TopologyBuilderException.class) public void shouldThrowExceptionWhenNoTopicPresent() throws Exception { - new KStreamBuilder().stream(); + builder.stream(); } @Test(expected = NullPointerException.class) public void shouldThrowExceptionWhenTopicNamesAreNull() throws Exception { - new KStreamBuilder().stream(Serdes.String(), Serdes.String(), null, null); + builder.stream(Serdes.String(), Serdes.String(), null, null); } @Test public void shouldNotMaterializeSourceKTableIfStateNameNotSpecified() throws Exception { - final KStreamBuilder builder = new KStreamBuilder(); - builder.setApplicationId("app-id"); - builder.table("topic1", "table1"); builder.table("topic2", null); - ProcessorTopology topology = builder.build(null); + final ProcessorTopology topology = builder.build(null); assertEquals(1, topology.stateStores().size()); assertEquals("table1", topology.stateStores().get(0).name()); @@ -162,23 +163,26 @@ public void shouldNotMaterializeSourceKTableIfStateNameNotSpecified() throws Exc @Test public void shouldBuildSimpleGlobalTableTopology() throws Exception { - final KStreamBuilder builder = new KStreamBuilder(); builder.globalTable("table", "globalTable"); + final ProcessorTopology topology = builder.buildGlobalStateTopology(); final List stateStores = topology.globalStateStores(); - final StateStore store = stateStores.iterator().next(); + assertEquals(1, stateStores.size()); - assertEquals("globalTable", store.name()); + assertEquals("globalTable", stateStores.get(0).name()); } @Test public void shouldBuildGlobalTopologyWithAllGlobalTables() throws Exception { - final KStreamBuilder builder = new KStreamBuilder(); builder.globalTable("table", "globalTable"); builder.globalTable("table2", "globalTable2"); + final ProcessorTopology topology = builder.buildGlobalStateTopology(); + final List stateStores = topology.globalStateStores(); - assertEquals(Utils.mkSet("table", "table2"), topology.sourceTopics()); + final Set sourceTopics = topology.sourceTopics(); + + assertEquals(Utils.mkSet("table", "table2"), sourceTopics); assertEquals(2, stateStores.size()); } @@ -186,7 +190,6 @@ public void shouldBuildGlobalTopologyWithAllGlobalTables() throws Exception { public void shouldAddGlobalTablesToEachGroup() throws Exception { final String one = "globalTable"; final String two = "globalTable2"; - final KStreamBuilder builder = new KStreamBuilder(); final GlobalKTable globalTable = builder.globalTable("table", one); final GlobalKTable globalTable2 = builder.globalTable("table2", two); @@ -203,7 +206,7 @@ public String apply(final String key, final String value) { stream.leftJoin(globalTable, kvMapper, MockValueJoiner.TOSTRING_JOINER); final KStream stream2 = builder.stream("t2"); stream2.leftJoin(globalTable2, kvMapper, MockValueJoiner.TOSTRING_JOINER); - builder.setApplicationId("app-id"); + final Map> nodeGroups = builder.nodeGroups(); for (Integer groupId : nodeGroups.keySet()) { final ProcessorTopology topology = builder.build(groupId); @@ -212,6 +215,7 @@ public String apply(final String key, final String value) { for (StateStore stateStore : stateStores) { names.add(stateStore.name()); } + assertEquals(2, stateStores.size()); assertTrue(names.contains(one)); assertTrue(names.contains(two)); @@ -220,9 +224,6 @@ public String apply(final String key, final String value) { @Test public void shouldMapStateStoresToCorrectSourceTopics() throws Exception { - final KStreamBuilder builder = new KStreamBuilder(); - builder.setApplicationId("app-id"); - final KStream playEvents = builder.stream("events"); final KTable table = builder.table("table-topic", "table-store"); @@ -230,7 +231,8 @@ public void shouldMapStateStoresToCorrectSourceTopics() throws Exception { final KStream mapped = playEvents.map(MockKeyValueMapper.SelectValueKeyValueMapper()); mapped.leftJoin(table, MockValueJoiner.TOSTRING_JOINER).groupByKey().count("count"); + assertEquals(Collections.singleton("table-topic"), builder.stateStoreNameToSourceTopics().get("table-store")); - assertEquals(Collections.singleton("app-id-KSTREAM-MAP-0000000003-repartition"), builder.stateStoreNameToSourceTopics().get("count")); + assertEquals(Collections.singleton(APP_ID + "-KSTREAM-MAP-0000000003-repartition"), builder.stateStoreNameToSourceTopics().get("count")); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java index 569ea5af0f921..b6988e31587e6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java @@ -141,6 +141,4 @@ public void testJoin() throws Exception { processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3"); } - - } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java index c3368a116a90f..2e5b201186199 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; -import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.Merger; import org.apache.kafka.streams.kstream.SessionWindows; import org.apache.kafka.streams.kstream.Windowed; @@ -32,7 +31,6 @@ import org.apache.kafka.streams.state.internals.RocksDBSessionStoreSupplier; import org.apache.kafka.streams.state.SessionStore; import org.apache.kafka.streams.state.internals.ThreadCache; -import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorContext; import org.apache.kafka.test.NoOpRecordCollector; import org.apache.kafka.test.TestUtils; @@ -91,7 +89,7 @@ public Long apply(final String aggKey, final Long aggOne, final Long aggTwo) { @Before public void initializeStore() { final File stateDir = TestUtils.tempDirectory(); - context = new MockProcessorContext(new KStreamTestDriver(new KStreamBuilder(), stateDir), stateDir, + context = new MockProcessorContext(stateDir, Serdes.String(), Serdes.String(), new NoOpRecordCollector(), new ThreadCache("testCache", 100000, new MockStreamsMetrics(new Metrics()))) { @Override public void forward(final K key, final V value) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java index c3df49d3222ea..efa0e0e629839 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java @@ -231,7 +231,7 @@ public void send(ProducerRecord record, Serializer keySeria - this.context = new MockProcessorContext(null, this.stateDir, serdes.keySerde(), serdes.valueSerde(), recordCollector, null) { + this.context = new MockProcessorContext(this.stateDir, serdes.keySerde(), serdes.valueSerde(), recordCollector, null) { @Override public TaskId taskId() { return new TaskId(0, 1); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java index 8746a86cb44b1..a00526f24c79a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java @@ -63,7 +63,7 @@ public void setUp() throws Exception { store = new CachingKeyValueStore<>(underlyingStore, Serdes.String(), Serdes.String()); store.setFlushListener(cacheFlushListener); cache = new ThreadCache("testCache", maxCacheSizeBytes, new MockStreamsMetrics(new Metrics())); - final MockProcessorContext context = new MockProcessorContext(null, null, null, null, (RecordCollector) null, cache); + final MockProcessorContext context = new MockProcessorContext(null, null, null, (RecordCollector) null, cache); topic = "topic"; context.setRecordContext(new ProcessorRecordContext(10, 0, 0, topic)); store.init(context, null); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java index 5035f70887e8d..65a249eb5c69c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java @@ -64,7 +64,7 @@ public void setUp() throws Exception { Serdes.String(), Serdes.Long()); cache = new ThreadCache("testCache", MAX_CACHE_SIZE_BYTES, new MockStreamsMetrics(new Metrics())); - final MockProcessorContext context = new MockProcessorContext(null, TestUtils.tempDirectory(), null, null, (RecordCollector) null, cache); + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), null, null, (RecordCollector) null, cache); context.setRecordContext(new ProcessorRecordContext(DEFAULT_TIMESTAMP, 0, 0, "topic")); cachingStore.init(context, cachingStore); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java index 1de1002d74c51..2728aa07ea67a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java @@ -69,7 +69,7 @@ public void setUp() throws Exception { cachingStore.setFlushListener(cacheListener); cache = new ThreadCache("testCache", MAX_CACHE_SIZE_BYTES, new MockStreamsMetrics(new Metrics())); topic = "topic"; - final MockProcessorContext context = new MockProcessorContext(null, TestUtils.tempDirectory(), null, null, (RecordCollector) null, cache); + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), null, null, (RecordCollector) null, cache); context.setRecordContext(new ProcessorRecordContext(DEFAULT_TIMESTAMP, 0, 0, topic)); cachingStore.init(context, cachingStore); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java index 82fb83123a095..99b13479dcdda 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java @@ -57,8 +57,7 @@ public void send(final ProducerRecord record, final Serializer k sent.put(record.key(), record.value()); } }; - final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.Long(), collector, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java index 8815c5a510e5f..442602ce91572 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java @@ -65,8 +65,7 @@ public void send(final ProducerRecord record, final Serializer k sent.put(record.key(), record.value()); } }; - final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.Long(), collector, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStoreTest.java index 621feb36c247b..51f31bf392e21 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStoreTest.java @@ -52,8 +52,7 @@ public void send(final ProducerRecord record, final Serializer k sent.put(record.key(), record.value()); } }; - final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.Long(), collector, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStoreTest.java index 63065129f14e1..9160a73a2f6c8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStoreTest.java @@ -93,8 +93,7 @@ public Sensor addSensor(String name, Sensor.RecordingLevel recordLevel, Sensor.. }; - final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.Long(), new NoOpRecordCollector(), diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplierTest.java index 3d9a56c271d6c..c51008931c084 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplierTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplierTest.java @@ -44,8 +44,7 @@ public class RocksDBKeyValueStoreSupplierTest { private static final String STORE_NAME = "name"; private final ThreadCache cache = new ThreadCache("test", 1024, new MockStreamsMetrics(new Metrics())); - private final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + private final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.String(), new NoOpRecordCollector(), @@ -67,8 +66,7 @@ public void send(final ProducerRecord record, final Serializer k logged.add(record); } }; - final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.String(), collector, @@ -89,8 +87,7 @@ public void send(final ProducerRecord record, final Serializer k logged.add(record); } }; - final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.String(), collector, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStoreTest.java index 7fe490c2b82f2..3763290765e85 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStoreTest.java @@ -62,8 +62,7 @@ public void before() { new SessionKeySchema()); stateDir = TestUtils.tempDirectory(); - final MockProcessorContext context = new MockProcessorContext(null, - stateDir, + final MockProcessorContext context = new MockProcessorContext(stateDir, Serdes.String(), Serdes.Long(), new NoOpRecordCollector(), diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplierTest.java index 28196a2a6ef8f..6677624e59627 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplierTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplierTest.java @@ -46,8 +46,7 @@ public class RocksDBSessionStoreSupplierTest { private static final String STORE_NAME = "name"; private final ThreadCache cache = new ThreadCache("test", 1024, new MockStreamsMetrics(new Metrics())); - private final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + private final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.String(), new NoOpRecordCollector(), @@ -70,8 +69,7 @@ public void send(final ProducerRecord record, final Serializer k logged.add(record); } }; - final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.String(), collector, @@ -92,8 +90,7 @@ public void send(final ProducerRecord record, final Serializer k logged.add(record); } }; - final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.String(), collector, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java index 5a23a1cb6740c..e1801b87cc04e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java @@ -53,8 +53,7 @@ public void before() { Serdes.String(), Serdes.Long()); - final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.Long(), new NoOpRecordCollector(), diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplierTest.java index 897ec623dc479..d9a0d4f3554fb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplierTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplierTest.java @@ -45,8 +45,7 @@ public class RocksDBWindowStoreSupplierTest { private static final String STORE_NAME = "name"; private WindowStore store; private final ThreadCache cache = new ThreadCache("test", 1024, new MockStreamsMetrics(new Metrics())); - private final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + private final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.String(), new NoOpRecordCollector(), @@ -67,8 +66,7 @@ public void send(final ProducerRecord record, final Serializer k logged.add(record); } }; - final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.String(), collector, @@ -89,8 +87,7 @@ public void send(final ProducerRecord record, final Serializer k logged.add(record); } }; - final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.String(), collector, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java index 79223dee81ab3..ee846f7383c10 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java @@ -100,7 +100,7 @@ public void send(ProducerRecord record, Serializer keySeria }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -151,7 +151,7 @@ public void send(ProducerRecord record, Serializer keySeria }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -225,7 +225,7 @@ public void send(ProducerRecord record, Serializer keySeria }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -314,7 +314,7 @@ public void send(ProducerRecord record, Serializer keySeria }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -401,7 +401,7 @@ public void send(ProducerRecord record, Serializer keySeria }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -457,7 +457,7 @@ public void send(ProducerRecord record, Serializer keySeria }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -485,7 +485,7 @@ public void send(ProducerRecord record, Serializer keySeria }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -613,7 +613,7 @@ public void send(ProducerRecord record, Serializer keySeria }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -662,7 +662,7 @@ public void send(ProducerRecord record, Serializer keySeria }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -710,7 +710,7 @@ public void send(ProducerRecord record, Serializer keySeria }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -810,7 +810,7 @@ public void send(ProducerRecord record, Serializer keySeria }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -870,7 +870,7 @@ public void send(ProducerRecord record, Serializer keySeria }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java index 3d2da31be0c52..ae6fb5a7b6581 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentIteratorTest.java @@ -48,14 +48,16 @@ public boolean hasNext(final KeyValueIterator iterator) { } }; + private SegmentIterator iterator = null; + @Before public void before() { - final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), - Serdes.String(), - Serdes.String(), - new NoOpRecordCollector(), - new ThreadCache("testCache", 0, new MockStreamsMetrics(new Metrics()))); + final MockProcessorContext context = new MockProcessorContext( + TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + new NoOpRecordCollector(), + new ThreadCache("testCache", 0, new MockStreamsMetrics(new Metrics()))); segmentOne.openDB(context); segmentTwo.openDB(context); segmentOne.put(Bytes.wrap("a".getBytes()), "1".getBytes()); @@ -67,13 +69,17 @@ public void before() { @After public void closeSegments() { + if (iterator != null) { + iterator.close(); + iterator = null; + } segmentOne.close(); segmentTwo.close(); } @Test public void shouldIterateOverAllSegments() throws Exception { - final SegmentIterator iterator = new SegmentIterator( + iterator = new SegmentIterator( Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, @@ -101,7 +107,7 @@ public void shouldIterateOverAllSegments() throws Exception { @Test public void shouldOnlyIterateOverSegmentsInRange() throws Exception { - final SegmentIterator iterator = new SegmentIterator( + iterator = new SegmentIterator( Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, @@ -121,7 +127,7 @@ public void shouldOnlyIterateOverSegmentsInRange() throws Exception { @Test(expected = NoSuchElementException.class) public void shouldThrowNoSuchElementOnPeekNextKeyIfNoNext() throws Exception { - final SegmentIterator iterator = new SegmentIterator( + iterator = new SegmentIterator( Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, @@ -133,7 +139,7 @@ public void shouldThrowNoSuchElementOnPeekNextKeyIfNoNext() throws Exception { @Test(expected = NoSuchElementException.class) public void shouldThrowNoSuchElementOnNextIfNoNext() throws Exception { - final SegmentIterator iterator = new SegmentIterator( + iterator = new SegmentIterator( Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, @@ -146,5 +152,4 @@ public void shouldThrowNoSuchElementOnNextIfNoNext() throws Exception { private KeyValue toStringKeyValue(final KeyValue binaryKv) { return KeyValue.pair(new String(binaryKv.key.get()), new String(binaryKv.value)); } - } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentsTest.java index 47207ec149521..9e34e63062be7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentsTest.java @@ -43,8 +43,7 @@ public class SegmentsTest { @Before public void createContext() { - context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.Long(), new NoOpRecordCollector(), diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java index d51384c12bc72..207705c89e45d 100644 --- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -42,14 +42,11 @@ public class KStreamTestDriver { + private static final long DEFAULT_CACHE_SIZE_BYTES = 1 * 1024 * 1024L; + private final ProcessorTopology topology; private final MockProcessorContext context; private final ProcessorTopology globalTopology; - private ThreadCache cache; - private static final long DEFAULT_CACHE_SIZE_BYTES = 1 * 1024 * 1024L; - public final File stateDir; - - private ProcessorNode currNode; public KStreamTestDriver(KStreamBuilder builder) { this(builder, null, Serdes.ByteArray(), Serdes.ByteArray()); @@ -78,9 +75,8 @@ public KStreamTestDriver(KStreamBuilder builder, builder.setApplicationId("TestDriver"); this.topology = builder.build(null); this.globalTopology = builder.buildGlobalStateTopology(); - this.stateDir = stateDir; - this.cache = new ThreadCache("testCache", cacheSize, new MockStreamsMetrics(new Metrics())); - this.context = new MockProcessorContext(this, stateDir, keySerde, valSerde, new MockRecordCollector(), cache); + ThreadCache cache = new ThreadCache("testCache", cacheSize, new MockStreamsMetrics(new Metrics())); + this.context = new MockProcessorContext(stateDir, keySerde, valSerde, new MockRecordCollector(), cache); this.context.setRecordContext(new ProcessorRecordContext(0, 0, 0, "topic")); // init global topology first as it will add stores to the // store map that are required for joins etc. @@ -88,7 +84,6 @@ public KStreamTestDriver(KStreamBuilder builder, initTopology(globalTopology, globalTopology.globalStateStores()); } initTopology(topology, topology.stateStores()); - } private void initTopology(final ProcessorTopology topology, final List stores) { @@ -106,14 +101,17 @@ private void initTopology(final ProcessorTopology topology, final List void forward(K key, V value) { - ProcessorNode thisNode = currNode; - for (ProcessorNode childNode : (List>) currNode.children()) { - currNode = childNode; - try { - childNode.process(key, value); - } finally { - currNode = thisNode; - } - } - } - - @SuppressWarnings("unchecked") - public void forward(K key, V value, int childIndex) { - ProcessorNode thisNode = currNode; - ProcessorNode childNode = (ProcessorNode) thisNode.children().get(childIndex); - currNode = childNode; - try { - childNode.process(key, value); - } finally { - currNode = thisNode; - } - } - - @SuppressWarnings("unchecked") - public void forward(K key, V value, String childName) { - ProcessorNode thisNode = currNode; - for (ProcessorNode childNode : (List>) thisNode.children()) { - if (childNode.name().equals(childName)) { - currNode = childNode; - try { - childNode.process(key, value); - } finally { - currNode = thisNode; - } - break; - } - } - } - public void close() { // close all processors for (ProcessorNode node : topology.processors()) { - currNode = node; + context.setCurrentNode(node); try { node.close(); } finally { - currNode = null; + context.setCurrentNode(null); } } - flushState(); + closeState(); } public Set allProcessorNames() { @@ -245,24 +196,19 @@ public void flushState() { } } - public void setCurrentNode(final ProcessorNode currentNode) { - currNode = currentNode; - } - - public StateStore globalStateStore(final String storeName) { - if (globalTopology != null) { - for (final StateStore store : globalTopology.globalStateStores()) { - if (store.name().equals(storeName)) { - return store; - } - } + private void closeState() { + for (StateStore stateStore : context.allStateStores().values()) { + stateStore.flush(); + stateStore.close(); } - return null; } + private ProcessorRecordContext createRecordContext(long timestamp) { + return new ProcessorRecordContext(timestamp, -1, -1, "topic"); + } private class MockRecordCollector extends RecordCollectorImpl { - public MockRecordCollector() { + MockRecordCollector() { super(null, "KStreamTestDriver"); } @@ -280,12 +226,9 @@ public void send(ProducerRecord record, Serializer keySerializer } @Override - public void flush() { - } + public void flush() {} @Override - public void close() { - } + public void close() {} } - } diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 5ae71124e3bd3..93f0f42370306 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -18,7 +18,6 @@ package org.apache.kafka.test; import java.io.File; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -48,7 +47,6 @@ public class MockProcessorContext implements InternalProcessorContext, RecordCollector.Supplier { - private final KStreamTestDriver driver; private final Serde keySerde; private final Serde valSerde; private final RecordCollector.Supplier recordCollectorSupplier; @@ -67,34 +65,34 @@ public class MockProcessorContext implements InternalProcessorContext, RecordCol private ProcessorNode currentNode; public MockProcessorContext(StateSerdes serdes, RecordCollector collector) { - this(null, null, serdes.keySerde(), serdes.valueSerde(), collector, null); + this(null, serdes.keySerde(), serdes.valueSerde(), collector, null); } - public MockProcessorContext(KStreamTestDriver driver, File stateDir, + public MockProcessorContext(File stateDir, Serde keySerde, Serde valSerde, final RecordCollector collector, final ThreadCache cache) { - this(driver, stateDir, keySerde, valSerde, + this(stateDir, keySerde, valSerde, new RecordCollector.Supplier() { @Override public RecordCollector recordCollector() { return collector; } - }, cache); + }, + cache); } - public MockProcessorContext(KStreamTestDriver driver, File stateDir, - Serde keySerde, - Serde valSerde, - RecordCollector.Supplier collectorSupplier, + public MockProcessorContext(final File stateDir, + final Serde keySerde, + final Serde valSerde, + final RecordCollector.Supplier collectorSupplier, final ThreadCache cache) { - this.driver = driver; this.stateDir = stateDir; this.keySerde = keySerde; this.valSerde = valSerde; this.recordCollectorSupplier = collectorSupplier; - this.metrics = new Metrics(config, Arrays.asList((MetricsReporter) new JmxReporter()), time, true); + this.metrics = new Metrics(config, Collections.singletonList((MetricsReporter) new JmxReporter()), time, true); this.cache = cache; this.streamsMetrics = new MockStreamsMetrics(metrics); } @@ -182,19 +180,45 @@ public void schedule(long interval) { @Override @SuppressWarnings("unchecked") public void forward(K key, V value) { - driver.forward(key, value); + ProcessorNode thisNode = currentNode; + for (ProcessorNode childNode : (List>) thisNode.children()) { + currentNode = childNode; + try { + childNode.process(key, value); + } finally { + currentNode = thisNode; + } + } } @Override @SuppressWarnings("unchecked") public void forward(K key, V value, int childIndex) { - driver.forward(key, value, childIndex); + ProcessorNode thisNode = currentNode; + ProcessorNode childNode = (ProcessorNode) thisNode.children().get(childIndex); + currentNode = childNode; + try { + childNode.process(key, value); + } finally { + currentNode = thisNode; + } } @Override @SuppressWarnings("unchecked") public void forward(K key, V value, String childName) { - driver.forward(key, value, childName); + ProcessorNode thisNode = currentNode; + for (ProcessorNode childNode : (List>) thisNode.children()) { + if (childNode.name().equals(childName)) { + currentNode = childNode; + try { + childNode.process(key, value); + } finally { + currentNode = thisNode; + } + break; + } + } } @@ -268,8 +292,7 @@ public void setRecordContext(final RecordContext recordContext) { @Override public void setCurrentNode(final ProcessorNode currentNode) { - this.currentNode = currentNode; - driver.setCurrentNode(currentNode); + this.currentNode = currentNode; } @Override From 919548ba0e1d0e231008c381f824f91a8924dc6a Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Wed, 18 Jan 2017 11:05:58 +0000 Subject: [PATCH 010/177] KAFKA-4568; Simplify test code for multiple SASL mechanisms Remove workaround for testing multiple SASL mechanisms using sasl.jaas.config and the new support for multiple client modules within a JVM. Author: Rajini Sivaram Reviewers: Sriharsha Chintalapani , Ismael Juma Closes #2373 from rajinisivaram/KAFKA-4568 (cherry picked from commit 4c492975a05e4a646f7140c560ed403c48925331) Signed-off-by: Ismael Juma --- .../kafka/api/BaseProducerSendTest.scala | 4 +- .../kafka/api/EndToEndAuthorizationTest.scala | 2 +- .../kafka/api/EndToEndClusterIdTest.scala | 2 +- .../kafka/api/IntegrationTestHarness.scala | 10 ++--- .../kafka/api/PlaintextConsumerTest.scala | 2 +- .../api/SaslEndToEndAuthorizationTest.scala | 17 ++++----- .../api/SaslMultiMechanismConsumerTest.scala | 6 +-- .../api/SaslPlainPlaintextConsumerTest.scala | 3 +- .../integration/kafka/api/SaslSetup.scala | 37 ++++++++++++------- .../kafka/api/SaslTestHarness.scala | 8 +--- .../api/SslEndToEndAuthorizationTest.scala | 2 +- .../integration/kafka/api/UserQuotaTest.scala | 3 +- .../integration/KafkaServerTestHarness.scala | 3 +- .../unit/kafka/server/BaseRequestTest.scala | 2 +- .../server/SaslApiVersionsRequestTest.scala | 3 +- .../unit/kafka/utils/JaasTestUtils.scala | 14 +++---- 16 files changed, 64 insertions(+), 54 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 82409bbcd88fc..9ebc7e3bf3b87 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -43,7 +43,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { val numServers = 2 overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString) TestUtils.createBrokerConfigs(numServers, zkConnect, false, interBrokerSecurityProtocol = Some(securityProtocol), - trustStoreFile = trustStoreFile, saslProperties = saslProperties).map(KafkaConfig.fromProps(_, overridingProps)) + trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties).map(KafkaConfig.fromProps(_, overridingProps)) } private var consumer: KafkaConsumer[Array[Byte], Array[Byte]] = _ @@ -69,7 +69,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { protected def createProducer(brokerList: String, retries: Int = 0, lingerMs: Long = 0, props: Option[Properties] = None): KafkaProducer[Array[Byte],Array[Byte]] = { val producer = TestUtils.createNewProducer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile, - saslProperties = saslProperties, retries = retries, lingerMs = lingerMs, props = props) + saslProperties = clientSaslProperties, retries = retries, lingerMs = lingerMs, props = props) registerProducer(producer) } diff --git a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala index 3e391d3911490..3866cc14e847a 100644 --- a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala @@ -169,7 +169,7 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas maxBlockMs = 3000L, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, - saslProperties = this.saslProperties, + saslProperties = this.clientSaslProperties, props = Some(producerConfig)) } diff --git a/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala index 9e03e2781cf84..d885d9b2894f9 100644 --- a/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala +++ b/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala @@ -104,7 +104,7 @@ class EndToEndClusterIdTest extends KafkaServerTestHarness { override def generateConfigs() = { val cfgs = TestUtils.createBrokerConfigs(serverCount, zkConnect, interBrokerSecurityProtocol = Some(securityProtocol), - trustStoreFile = trustStoreFile, saslProperties = saslProperties) + trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties) cfgs.foreach(_.putAll(serverConfig)) cfgs.map(KafkaConfig.fromProps) } diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index ee556d71d94da..92088f8210cc0 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -47,15 +47,15 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { override def generateConfigs() = { val cfgs = TestUtils.createBrokerConfigs(serverCount, zkConnect, interBrokerSecurityProtocol = Some(securityProtocol), - trustStoreFile = trustStoreFile, saslProperties = saslProperties) + trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties) cfgs.foreach(_.putAll(serverConfig)) cfgs.map(KafkaConfig.fromProps) } @Before override def setUp() { - val producerSecurityProps = TestUtils.producerSecurityConfigs(securityProtocol, trustStoreFile, saslProperties) - val consumerSecurityProps = TestUtils.consumerSecurityConfigs(securityProtocol, trustStoreFile, saslProperties) + val producerSecurityProps = TestUtils.producerSecurityConfigs(securityProtocol, trustStoreFile, clientSaslProperties) + val consumerSecurityProps = TestUtils.consumerSecurityConfigs(securityProtocol, trustStoreFile, clientSaslProperties) super.setUp() producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) @@ -81,7 +81,7 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { TestUtils.createNewProducer(brokerList, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, - saslProperties = this.saslProperties, + saslProperties = this.clientSaslProperties, props = Some(producerConfig)) } @@ -89,7 +89,7 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { TestUtils.createNewConsumer(brokerList, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, - saslProperties = this.saslProperties, + saslProperties = this.clientSaslProperties, props = Some(consumerConfig)) } diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index aefe5bd15ba25..282d67cf5c741 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -528,7 +528,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { producerProps.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, CompressionType.GZIP.name) producerProps.setProperty(ProducerConfig.LINGER_MS_CONFIG, Long.MaxValue.toString) val producer = TestUtils.createNewProducer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile, - saslProperties = saslProperties, retries = 0, lingerMs = Long.MaxValue, props = Some(producerProps)) + saslProperties = clientSaslProperties, retries = 0, lingerMs = Long.MaxValue, props = Some(producerProps)) (0 until numRecords).foreach { i => producer.send(new ProducerRecord(tp.topic, tp.partition, i.toLong, s"key $i".getBytes, s"value $i".getBytes)) } diff --git a/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala index 992649a0f9eff..826eb5ce8babf 100644 --- a/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala @@ -16,10 +16,9 @@ */ package kafka.api -import java.io.File import java.util.Properties -import kafka.utils.{JaasTestUtils,TestUtils} +import kafka.utils.TestUtils import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.config.SaslConfigs import org.apache.kafka.common.errors.GroupAuthorizationException @@ -30,24 +29,24 @@ import scala.collection.JavaConverters._ abstract class SaslEndToEndAuthorizationTest extends EndToEndAuthorizationTest { override protected def securityProtocol = SecurityProtocol.SASL_SSL - override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms))) + override protected val serverSaslProperties = Some(kafkaServerSaslProperties(kafkaServerSaslMechanisms, kafkaClientSaslMechanism)) + override protected val clientSaslProperties = Some(kafkaClientSaslProperties(kafkaClientSaslMechanism)) protected def kafkaClientSaslMechanism: String protected def kafkaServerSaslMechanisms: List[String] @Before override def setUp { - startSasl(Both, List(kafkaClientSaslMechanism), kafkaServerSaslMechanisms) + startSasl(Both, kafkaServerSaslMechanisms, Option(kafkaClientSaslMechanism)) super.setUp } // Use JAAS configuration properties for clients so that dynamic JAAS configuration is also tested by this set of tests - override protected def setJaasConfiguration(mode: SaslSetupMode, serverMechanisms: List[String], clientMechanisms: List[String], - serverKeytabFile: Option[File] = None, clientKeytabFile: Option[File] = None) { + override protected def setJaasConfiguration(mode: SaslSetupMode, serverMechanisms: List[String], clientMechanism: Option[String]) { // create static config with client login context with credentials for JaasTestUtils 'client2' - super.setJaasConfiguration(mode, kafkaServerSaslMechanisms, clientMechanisms, serverKeytabFile, clientKeytabFile) + super.setJaasConfiguration(mode, kafkaServerSaslMechanisms, clientMechanism) // set dynamic properties with credentials for JaasTestUtils 'client1' - val clientLoginContext = JaasTestUtils.clientLoginModule(kafkaClientSaslMechanism, clientKeytabFile) + val clientLoginContext = jaasClientLoginModule(kafkaClientSaslMechanism) producerConfig.put(SaslConfigs.SASL_JAAS_CONFIG, clientLoginContext) consumerConfig.put(SaslConfigs.SASL_JAAS_CONFIG, clientLoginContext) } @@ -70,7 +69,7 @@ abstract class SaslEndToEndAuthorizationTest extends EndToEndAuthorizationTest { val consumer2 = TestUtils.createNewConsumer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile, - saslProperties = saslProperties, + saslProperties = clientSaslProperties, props = Some(consumer2Config)) consumers += consumer2 diff --git a/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala index 5814e9463560c..3ff133f228ac4 100644 --- a/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala @@ -23,11 +23,11 @@ class SaslMultiMechanismConsumerTest extends BaseConsumerTest with SaslTestHarne override protected val zkSaslEnabled = true override protected val kafkaClientSaslMechanism = "PLAIN" override protected val kafkaServerSaslMechanisms = List("GSSAPI", "PLAIN") - override protected def allKafkaClientSaslMechanisms = List("PLAIN", "GSSAPI") this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "true") override protected def securityProtocol = SecurityProtocol.SASL_SSL override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) - override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms))) + override protected val serverSaslProperties = Some(kafkaServerSaslProperties(kafkaServerSaslMechanisms, kafkaClientSaslMechanism)) + override protected val clientSaslProperties = Some(kafkaClientSaslProperties(kafkaClientSaslMechanism)) @Test def testMultipleBrokerMechanisms() { @@ -35,7 +35,7 @@ class SaslMultiMechanismConsumerTest extends BaseConsumerTest with SaslTestHarne val plainSaslProducer = producers.head val plainSaslConsumer = consumers.head - val gssapiSaslProperties = kafkaSaslProperties("GSSAPI") + val gssapiSaslProperties = kafkaClientSaslProperties("GSSAPI", dynamicJaasConfig = true) val gssapiSaslProducer = TestUtils.createNewProducer(brokerList, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, diff --git a/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala index bdca577c0e878..125d4318ebe8b 100644 --- a/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala @@ -23,5 +23,6 @@ class SaslPlainPlaintextConsumerTest extends BaseConsumerTest with SaslTestHarne this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "true") override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) - override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms))) + override protected val serverSaslProperties = Some(kafkaServerSaslProperties(kafkaServerSaslMechanisms, kafkaClientSaslMechanism)) + override protected val clientSaslProperties = Some(kafkaClientSaslProperties(kafkaClientSaslMechanism)) } diff --git a/core/src/test/scala/integration/kafka/api/SaslSetup.scala b/core/src/test/scala/integration/kafka/api/SaslSetup.scala index c1e2da2cbf797..36b9d41e0584d 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSetup.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSetup.scala @@ -43,32 +43,36 @@ trait SaslSetup { private val workDir = TestUtils.tempDir() private val kdcConf = MiniKdc.createConfig private var kdc: MiniKdc = null + private var serverKeytabFile: Option[File] = null + private var clientKeytabFile: Option[File] = null - def startSasl(mode: SaslSetupMode = Both, kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanisms: List[String]) { + def startSasl(mode: SaslSetupMode = Both, kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanism: Option[String]) { // Important if tests leak consumers, producers or brokers LoginManager.closeAll() - val hasKerberos = mode != ZkSasl && (kafkaClientSaslMechanisms.contains("GSSAPI") || kafkaServerSaslMechanisms.contains("GSSAPI")) + val hasKerberos = mode != ZkSasl && (kafkaClientSaslMechanism == Some("GSSAPI") || kafkaServerSaslMechanisms.contains("GSSAPI")) if (hasKerberos) { val serverKeytabFile = TestUtils.tempFile() val clientKeytabFile = TestUtils.tempFile() - setJaasConfiguration(mode, kafkaServerSaslMechanisms, kafkaClientSaslMechanisms, Some(serverKeytabFile), Some(clientKeytabFile)) + this.clientKeytabFile = Some(clientKeytabFile) + this.serverKeytabFile = Some(serverKeytabFile) kdc = new MiniKdc(kdcConf, workDir) kdc.start() kdc.createPrincipal(serverKeytabFile, JaasTestUtils.KafkaServerPrincipalUnqualifiedName + "/localhost") kdc.createPrincipal(clientKeytabFile, JaasTestUtils.KafkaClientPrincipalUnqualifiedName, JaasTestUtils.KafkaClientPrincipalUnqualifiedName2) } else { - setJaasConfiguration(mode, kafkaServerSaslMechanisms, kafkaClientSaslMechanisms) + this.clientKeytabFile = None + this.serverKeytabFile = None } + setJaasConfiguration(mode, kafkaServerSaslMechanisms, kafkaClientSaslMechanism) if (mode == Both || mode == ZkSasl) System.setProperty("zookeeper.authProvider.1", "org.apache.zookeeper.server.auth.SASLAuthenticationProvider") } - protected def setJaasConfiguration(mode: SaslSetupMode, kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanisms: List[String], - serverKeytabFile: Option[File] = None, clientKeytabFile: Option[File] = None) { + protected def setJaasConfiguration(mode: SaslSetupMode, kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanism: Option[String]) { val jaasFile = mode match { case ZkSasl => JaasTestUtils.writeZkFile() - case KafkaSasl => JaasTestUtils.writeKafkaFile(kafkaServerSaslMechanisms, kafkaClientSaslMechanisms, serverKeytabFile, clientKeytabFile) - case Both => JaasTestUtils.writeZkAndKafkaFiles(kafkaServerSaslMechanisms, kafkaClientSaslMechanisms, serverKeytabFile, clientKeytabFile) + case KafkaSasl => JaasTestUtils.writeKafkaFile(kafkaServerSaslMechanisms, kafkaClientSaslMechanism, serverKeytabFile, clientKeytabFile) + case Both => JaasTestUtils.writeZkAndKafkaFiles(kafkaServerSaslMechanisms, kafkaClientSaslMechanism, serverKeytabFile, clientKeytabFile) } // This will cause a reload of the Configuration singleton when `getConfiguration` is called Configuration.setConfiguration(null) @@ -85,13 +89,20 @@ trait SaslSetup { Configuration.setConfiguration(null) } - def kafkaSaslProperties(clientSaslMechanism: String, serverSaslMechanisms: Option[Seq[String]] = None) = { + def kafkaServerSaslProperties(serverSaslMechanisms: Seq[String], interBrokerSaslMechanism: String) = { + val props = new Properties + props.put(KafkaConfig.SaslMechanismInterBrokerProtocolProp, interBrokerSaslMechanism) + props.put(SaslConfigs.SASL_ENABLED_MECHANISMS, serverSaslMechanisms.mkString(",")) + props + } + + def kafkaClientSaslProperties(clientSaslMechanism: String, dynamicJaasConfig: Boolean = false) = { val props = new Properties props.put(SaslConfigs.SASL_MECHANISM, clientSaslMechanism) - serverSaslMechanisms.foreach { serverMechanisms => - props.put(KafkaConfig.SaslMechanismInterBrokerProtocolProp, clientSaslMechanism) - props.put(SaslConfigs.SASL_ENABLED_MECHANISMS, serverMechanisms.mkString(",")) - } + if (dynamicJaasConfig) + props.put(SaslConfigs.SASL_JAAS_CONFIG, jaasClientLoginModule(clientSaslMechanism)) props } + + def jaasClientLoginModule(clientSaslMechanism: String): String = JaasTestUtils.clientLoginModule(clientSaslMechanism, clientKeytabFile) } diff --git a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala index 8fd3eb4422e0e..97faa36e7fa2c 100644 --- a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala @@ -20,16 +20,12 @@ trait SaslTestHarness extends ZooKeeperTestHarness with SaslSetup { protected val kafkaClientSaslMechanism = "GSSAPI" protected val kafkaServerSaslMechanisms = List(kafkaClientSaslMechanism) - // Override this list to enable client login modules for multiple mechanisms for testing - // of multi-mechanism brokers with clients using different mechanisms in a single JVM - protected def allKafkaClientSaslMechanisms = List(kafkaClientSaslMechanism) - @Before override def setUp() { if (zkSaslEnabled) - startSasl(Both, kafkaServerSaslMechanisms, allKafkaClientSaslMechanisms) + startSasl(Both, kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism)) else - startSasl(KafkaSasl, kafkaServerSaslMechanisms, allKafkaClientSaslMechanisms) + startSasl(KafkaSasl, kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism)) super.setUp } diff --git a/core/src/test/scala/integration/kafka/api/SslEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/SslEndToEndAuthorizationTest.scala index 365c0ba5b5886..064e783508130 100644 --- a/core/src/test/scala/integration/kafka/api/SslEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SslEndToEndAuthorizationTest.scala @@ -29,7 +29,7 @@ class SslEndToEndAuthorizationTest extends EndToEndAuthorizationTest { @Before override def setUp { - startSasl(ZkSasl, List.empty, List.empty) + startSasl(ZkSasl, List.empty, None) super.setUp } } diff --git a/core/src/test/scala/integration/kafka/api/UserQuotaTest.scala b/core/src/test/scala/integration/kafka/api/UserQuotaTest.scala index 4677c8ce9ab2b..c8d0a77aac146 100644 --- a/core/src/test/scala/integration/kafka/api/UserQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/api/UserQuotaTest.scala @@ -29,7 +29,8 @@ class UserQuotaTest extends BaseQuotaTest with SaslTestHarness { override protected def securityProtocol = SecurityProtocol.SASL_SSL override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) override protected val zkSaslEnabled = false - override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms))) + override protected val serverSaslProperties = Some(kafkaServerSaslProperties(kafkaServerSaslMechanisms, kafkaClientSaslMechanism)) + override protected val clientSaslProperties = Some(kafkaClientSaslProperties(kafkaClientSaslMechanism)) override val userPrincipal = JaasTestUtils.KafkaClientPrincipalUnqualifiedName2 override val producerQuotaId = QuotaId(Some(userPrincipal), None) diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 270fca2e01624..bfaff0bb343e3 100755 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -75,7 +75,8 @@ trait KafkaServerTestHarness extends ZooKeeperTestHarness { protected def securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT protected def listenerName: ListenerName = ListenerName.forSecurityProtocol(securityProtocol) protected def trustStoreFile: Option[File] = None - protected def saslProperties: Option[Properties] = None + protected def serverSaslProperties: Option[Properties] = None + protected def clientSaslProperties: Option[Properties] = None @Before override def setUp() { diff --git a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala index 3825489ddb737..b864e5d0875fb 100644 --- a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala @@ -42,7 +42,7 @@ abstract class BaseRequestTest extends KafkaServerTestHarness { val props = TestUtils.createBrokerConfigs(numBrokers, zkConnect, enableControlledShutdown = false, enableDeleteTopic = true, interBrokerSecurityProtocol = Some(securityProtocol), - trustStoreFile = trustStoreFile, saslProperties = saslProperties) + trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties) props.foreach(propertyOverrides) props.map(KafkaConfig.fromProps) } diff --git a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala index 34d7d147c1cb7..07e03e3824ef9 100644 --- a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala @@ -33,7 +33,8 @@ class SaslApiVersionsRequestTest extends BaseRequestTest with SaslTestHarness { override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT override protected val kafkaClientSaslMechanism = "PLAIN" override protected val kafkaServerSaslMechanisms = List("PLAIN") - override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms))) + override protected val serverSaslProperties = Some(kafkaServerSaslProperties(kafkaServerSaslMechanisms, kafkaClientSaslMechanism)) + override protected val clientSaslProperties = Some(kafkaClientSaslProperties(kafkaClientSaslMechanism)) override protected val zkSaslEnabled = false override def numBrokers = 1 diff --git a/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala b/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala index 7055b7e2b835b..0949eb793b4e5 100644 --- a/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala @@ -128,16 +128,16 @@ object JaasTestUtils { jaasFile.getCanonicalPath } - def writeKafkaFile(kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanisms: List[String], serverKeyTabLocation: Option[File], clientKeyTabLocation: Option[File]): String = { + def writeKafkaFile(kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanism: Option[String], serverKeyTabLocation: Option[File], clientKeyTabLocation: Option[File]): String = { val jaasFile = TestUtils.tempFile() - val kafkaSections = Seq(kafkaServerSection(kafkaServerSaslMechanisms, serverKeyTabLocation), kafkaClientSection(kafkaClientSaslMechanisms, clientKeyTabLocation)) + val kafkaSections = Seq(kafkaServerSection(kafkaServerSaslMechanisms, serverKeyTabLocation), kafkaClientSection(kafkaClientSaslMechanism, clientKeyTabLocation)) writeToFile(jaasFile, kafkaSections) jaasFile.getCanonicalPath } - def writeZkAndKafkaFiles(kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanisms: List[String], serverKeyTabLocation: Option[File], clientKeyTabLocation: Option[File]): String = { + def writeZkAndKafkaFiles(kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanism: Option[String], serverKeyTabLocation: Option[File], clientKeyTabLocation: Option[File]): String = { val jaasFile = TestUtils.tempFile() - val kafkaSections = Seq(kafkaServerSection(kafkaServerSaslMechanisms, serverKeyTabLocation), kafkaClientSection(kafkaClientSaslMechanisms, clientKeyTabLocation)) + val kafkaSections = Seq(kafkaServerSection(kafkaServerSaslMechanisms, serverKeyTabLocation), kafkaClientSection(kafkaClientSaslMechanism, clientKeyTabLocation)) writeToFile(jaasFile, kafkaSections ++ zkSections) jaasFile.getCanonicalPath } @@ -209,9 +209,9 @@ object JaasTestUtils { /* * Used for the static JAAS configuration and it uses the credentials for client#2 */ - private def kafkaClientSection(mechanisms: List[String], keytabLocation: Option[File]): JaasSection = { - new JaasSection(KafkaClientContextName, mechanisms.map(m => - kafkaClientModule(m, keytabLocation, KafkaClientPrincipal2, KafkaPlainUser2, KafkaPlainPassword2, KafkaScramUser2, KafkaScramPassword2))) + private def kafkaClientSection(mechanism: Option[String], keytabLocation: Option[File]): JaasSection = { + new JaasSection(KafkaClientContextName, mechanism.map(m => + kafkaClientModule(m, keytabLocation, KafkaClientPrincipal2, KafkaPlainUser2, KafkaPlainPassword2, KafkaScramUser2, KafkaScramPassword2)).toSeq) } private def jaasSectionsToString(jaasSections: Seq[JaasSection]): String = From 9b751f0c54e0409380632c4ee4618d78daca87d5 Mon Sep 17 00:00:00 2001 From: Damian Guy Date: Wed, 18 Jan 2017 15:17:05 +0000 Subject: [PATCH 011/177] MINOR: Fix javadoc typos in KStream#process interface for `Processor` in comments incorrectly had `transform` rather than `process`. Author: Damian Guy Reviewers: Michael G. Noll, Ismael Juma Closes #2396 from dguy/minor-javadoc (cherry picked from commit ccb183f9fca20a37b9b39761dedcd197e3cf2033) Signed-off-by: Ismael Juma --- .../main/java/org/apache/kafka/streams/kstream/KStream.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 8bc54e2256ab7..3e95e7cec0d26 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -706,7 +706,7 @@ KStream transformValues(final ValueTransformerSupplier * In order to assign a state, the state must be created and registered beforehand: @@ -739,7 +739,7 @@ KStream transformValues(final ValueTransformerSupplier Date: Wed, 18 Jan 2017 09:19:06 -0800 Subject: [PATCH 012/177] HOTFIX: ChangeLoggingKeyValueStore.name() returns null This class doesn't need to override this method as it is handled appropriately by the super class Author: Damian Guy Reviewers: Guozhang Wang Closes #2397 from dguy/hotfix-npe-state-store (cherry picked from commit 961ebca57fdd4f8f84df8cf26a835f07ea1718c9) Signed-off-by: Guozhang Wang --- .../state/internals/ChangeLoggingKeyValueStore.java | 7 +------ .../state/internals/ChangeLoggingKeyValueStoreTest.java | 5 +++++ 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStore.java index 11cf8022d8bb3..cd63d1ab30b75 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStore.java @@ -48,12 +48,7 @@ private ChangeLoggingKeyValueStore(final ChangeLoggingKeyValueBytesStore bytesSt this.keySerde = keySerde; this.valueSerde = valueSerde; } - - @Override - public String name() { - return null; - } - + @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context, final StateStore root) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java index 442602ce91572..2dc2017fd4862 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java @@ -200,6 +200,11 @@ public void shouldReturnNullOnGetWhenDoesntExist() throws Exception { assertThat(store.get(hello), is(nullValue())); } + @Test + public void shouldReturnInnerStoreName() throws Exception { + assertThat(store.name(), equalTo("kv")); + } + private String deserializedValueFromInner(final String key) { return valueSerde.deserializer().deserialize("blah", inner.get(Bytes.wrap(key.getBytes()))); } From dbca4a3b60d6039558c6fec99c86400c2131f9e8 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Wed, 18 Jan 2017 11:55:23 -0800 Subject: [PATCH 013/177] MINOR: refactor streams system test class hierachy Author: Matthias J. Sax Reviewers: Eno Thereska, Guozhang Wang Closes #2392 from mjsax/minor-system-test-rework (cherry picked from commit d8a77560c2fa2c209353e3ba2366ad3d4cfdf22c) Signed-off-by: Guozhang Wang --- .../kafka/streams/perf/SimpleBenchmark.java | 2 +- .../performance/streams_performance.py | 112 ++---------------- tests/kafkatest/services/streams.py | 54 +++++---- .../streams/streams_shutdown_deadlock_test.py | 3 +- 4 files changed, 43 insertions(+), 128 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java index 7ba61613d43cb..db3752db46961 100644 --- a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java +++ b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java @@ -111,7 +111,7 @@ public static void main(String[] args) throws Exception { rocksdbDir.mkdir(); // Note: this output is needed for automated tests and must not be removed - System.out.println("SimpleBenchmark instance started"); + System.out.println("StreamsTest instance started"); System.out.println("kafka=" + kafka); System.out.println("zookeeper=" + zookeeper); System.out.println("stateDir=" + stateDir); diff --git a/tests/kafkatest/services/performance/streams_performance.py b/tests/kafkatest/services/performance/streams_performance.py index 0af13f9b349ca..e9fa2a79acd40 100644 --- a/tests/kafkatest/services/performance/streams_performance.py +++ b/tests/kafkatest/services/performance/streams_performance.py @@ -13,115 +13,17 @@ # See the License for the specific language governing permissions and # limitations under the License. -import os.path -import signal +from kafkatest.services.streams import StreamsTestBaseService -from ducktape.services.service import Service -from ducktape.utils.util import wait_until - -from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin # # Class used to start the simple Kafka Streams benchmark # -class StreamsSimpleBenchmarkService(KafkaPathResolverMixin, Service): +class StreamsSimpleBenchmarkService(StreamsTestBaseService): """Base class for simple Kafka Streams benchmark""" - PERSISTENT_ROOT = "/mnt/streams" - # The log file contains normal log4j logs written using a file appender. stdout and stderr are handled separately - LOG_FILE = os.path.join(PERSISTENT_ROOT, "streams.log") - STDOUT_FILE = os.path.join(PERSISTENT_ROOT, "streams.stdout") - STDERR_FILE = os.path.join(PERSISTENT_ROOT, "streams.stderr") - LOG4J_CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties") - PID_FILE = os.path.join(PERSISTENT_ROOT, "streams.pid") - - logs = { - "streams_log": { - "path": LOG_FILE, - "collect_default": True}, - "streams_stdout": { - "path": STDOUT_FILE, - "collect_default": True}, - "streams_stderr": { - "path": STDERR_FILE, - "collect_default": True}, - } - - def __init__(self, context, kafka, numrecs): - super(StreamsSimpleBenchmarkService, self).__init__(context, 1) - self.kafka = kafka - self.numrecs = numrecs - - @property - def node(self): - return self.nodes[0] - - def pids(self, node): - try: - return [pid for pid in node.account.ssh_capture("cat " + self.PID_FILE, callback=int)] - except: - return [] - - def stop_node(self, node, clean_shutdown=True): - self.logger.info((clean_shutdown and "Cleanly" or "Forcibly") + " stopping SimpleBenchmark on " + str(node.account)) - pids = self.pids(node) - sig = signal.SIGTERM if clean_shutdown else signal.SIGKILL - - for pid in pids: - node.account.signal(pid, sig, allow_fail=True) - if clean_shutdown: - for pid in pids: - wait_until(lambda: not node.account.alive(pid), timeout_sec=60, err_msg="SimpleBenchmark process on " + str(node.account) + " took too long to exit") - - node.account.ssh("rm -f " + self.PID_FILE, allow_fail=False) - - def wait(self): - for node in self.nodes: - for pid in self.pids(node): - wait_until(lambda: not node.account.alive(pid), timeout_sec=600, backoff_sec=1, err_msg="SimpleBenchmark process on " + str(node.account) + " took too long to exit") - - def clean_node(self, node): - node.account.kill_process("streams", clean_shutdown=False, allow_fail=True) - node.account.ssh("rm -rf " + self.PERSISTENT_ROOT, allow_fail=False) - - def start_cmd(self, node): - args = {} - args['kafka'] = self.kafka.bootstrap_servers() - args['zk'] = self.kafka.zk.connect_setting() - args['state_dir'] = self.PERSISTENT_ROOT - args['numrecs'] = self.numrecs - args['stdout'] = self.STDOUT_FILE - args['stderr'] = self.STDERR_FILE - args['pidfile'] = self.PID_FILE - args['log4j'] = self.LOG4J_CONFIG_FILE - args['kafka_run_class'] = self.path.script("kafka-run-class.sh", node) - - cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%(log4j)s\"; " \ - "INCLUDE_TEST_JARS=true %(kafka_run_class)s org.apache.kafka.streams.perf.SimpleBenchmark " \ - " %(kafka)s %(zk)s %(state_dir)s %(numrecs)s " \ - " & echo $! >&3 ) 1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % args - - return cmd - - def start_node(self, node): - node.account.ssh("mkdir -p %s" % self.PERSISTENT_ROOT, allow_fail=False) - - node.account.create_file(self.LOG4J_CONFIG_FILE, self.render('tools_log4j.properties', log_file=self.LOG_FILE)) - - self.logger.info("Starting SimpleBenchmark process on " + str(node.account)) - results = {} - with node.account.monitor_log(self.STDOUT_FILE) as monitor: - node.account.ssh(self.start_cmd(node)) - monitor.wait_until('SimpleBenchmark instance started', timeout_sec=15, err_msg="Never saw message indicating SimpleBenchmark finished startup on " + str(node.account)) - - if len(self.pids(node)) == 0: - raise RuntimeError("No process ids recorded") - - def collect_data(self, node): - # Collect the data and return it to the framework - output = node.account.ssh_capture("grep Performance %s" % self.STDOUT_FILE) - data = {} - for line in output: - parts = line.split(':') - data[parts[0]] = float(parts[1]) - return data + def __init__(self, test_context, kafka, numrecs): + super(StreamsSimpleBenchmarkService, self).__init__(test_context, + kafka, + "org.apache.kafka.streams.perf.SimpleBenchmark", + numrecs) diff --git a/tests/kafkatest/services/streams.py b/tests/kafkatest/services/streams.py index 87e441470e425..9250cd7b94375 100644 --- a/tests/kafkatest/services/streams.py +++ b/tests/kafkatest/services/streams.py @@ -22,8 +22,8 @@ from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin -class StreamsSmokeTestBaseService(KafkaPathResolverMixin, Service): - """Base class for Streams Smoke Test services providing some common settings and functionality""" +class StreamsTestBaseService(KafkaPathResolverMixin, Service): + """Base class for Streams Test services providing some common settings and functionality""" PERSISTENT_ROOT = "/mnt/streams" # The log file contains normal log4j logs written using a file appender. stdout and stderr are handled separately @@ -45,10 +45,11 @@ class StreamsSmokeTestBaseService(KafkaPathResolverMixin, Service): "collect_default": True}, } - def __init__(self, context, kafka, command): - super(StreamsSmokeTestBaseService, self).__init__(context, 1) + def __init__(self, test_context, kafka, streams_class_name, user_test_args): + super(StreamsTestBaseService, self).__init__(test_context, 1) self.kafka = kafka - self.args = {'command': command} + self.args = {'streams_class_name': streams_class_name, + 'user_test_args': user_test_args} @property def node(self): @@ -65,7 +66,7 @@ def stop_nodes(self, clean_shutdown=True): self.stop_node(node, clean_shutdown) def stop_node(self, node, clean_shutdown=True): - self.logger.info((clean_shutdown and "Cleanly" or "Forcibly") + " stopping Streams Smoke Test on " + str(node.account)) + self.logger.info((clean_shutdown and "Cleanly" or "Forcibly") + " stopping Streams Test on " + str(node.account)) pids = self.pids(node) sig = signal.SIGTERM if clean_shutdown else signal.SIGKILL @@ -73,7 +74,7 @@ def stop_node(self, node, clean_shutdown=True): node.account.signal(pid, sig, allow_fail=True) if clean_shutdown: for pid in pids: - wait_until(lambda: not node.account.alive(pid), timeout_sec=60, err_msg="Streams Smoke Test process on " + str(node.account) + " took too long to exit") + wait_until(lambda: not node.account.alive(pid), timeout_sec=60, err_msg="Streams Test process on " + str(node.account) + " took too long to exit") node.account.ssh("rm -f " + self.PID_FILE, allow_fail=False) @@ -95,8 +96,11 @@ def abortThenRestart(self): def wait(self, timeout_sec=360): for node in self.nodes: - for pid in self.pids(node): - wait_until(lambda: not node.account.alive(pid), timeout_sec=timeout_sec, err_msg="Streams Smoke Test process on " + str(node.account) + " took too long to exit") + self.wait_node(node, timeout_sec) + + def wait_node(self, node, timeout_sec=None): + for pid in self.pids(node): + wait_until(lambda: not node.account.alive(pid), timeout_sec=timeout_sec, err_msg="Streams Test process on " + str(node.account) + " took too long to exit") def clean_node(self, node): node.account.kill_process("streams", clean_shutdown=False, allow_fail=True) @@ -105,7 +109,6 @@ def clean_node(self, node): def start_cmd(self, node): args = self.args.copy() args['kafka'] = self.kafka.bootstrap_servers() - args['zk'] = self.kafka.zk.connect_setting() args['state_dir'] = self.PERSISTENT_ROOT args['stdout'] = self.STDOUT_FILE args['stderr'] = self.STDERR_FILE @@ -114,8 +117,8 @@ def start_cmd(self, node): args['kafka_run_class'] = self.path.script("kafka-run-class.sh", node) cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%(log4j)s\"; " \ - "INCLUDE_TEST_JARS=true %(kafka_run_class)s org.apache.kafka.streams.smoketest.StreamsSmokeTest " \ - " %(command)s %(kafka)s %(zk)s %(state_dir)s " \ + "INCLUDE_TEST_JARS=true %(kafka_run_class)s %(streams_class_name)s " \ + " %(kafka)s %(state_dir)s %(user_test_args)s" \ " & echo $! >&3 ) 1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % args return cmd @@ -125,24 +128,35 @@ def start_node(self, node): node.account.create_file(self.LOG4J_CONFIG_FILE, self.render('tools_log4j.properties', log_file=self.LOG_FILE)) - self.logger.info("Starting StreamsSmokeTest process on " + str(node.account)) + self.logger.info("Starting StreamsTest process on " + str(node.account)) with node.account.monitor_log(self.STDOUT_FILE) as monitor: node.account.ssh(self.start_cmd(node)) - monitor.wait_until('StreamsSmokeTest instance started', timeout_sec=15, err_msg="Never saw message indicating StreamsSmokeTest finished startup on " + str(node.account)) + monitor.wait_until('StreamsTest instance started', timeout_sec=60, err_msg="Never saw message indicating StreamsTest finished startup on " + str(node.account)) if len(self.pids(node)) == 0: raise RuntimeError("No process ids recorded") +class StreamsSmokeTestBaseService(StreamsTestBaseService): + """Base class for Streams Smoke Test services providing some common settings and functionality""" + + def __init__(self, test_context, kafka, command): + super(StreamsSmokeTestBaseService, self).__init__(test_context, + kafka, + "org.apache.kafka.streams.smoketest.StreamsSmokeTest", + command) + + class StreamsSmokeTestDriverService(StreamsSmokeTestBaseService): - def __init__(self, context, kafka): - super(StreamsSmokeTestDriverService, self).__init__(context, kafka, "run") + def __init__(self, test_context, kafka): + super(StreamsSmokeTestDriverService, self).__init__(test_context, kafka, "run") class StreamsSmokeTestJobRunnerService(StreamsSmokeTestBaseService): - def __init__(self, context, kafka): - super(StreamsSmokeTestJobRunnerService, self).__init__(context, kafka, "process") + def __init__(self, test_context, kafka): + super(StreamsSmokeTestJobRunnerService, self).__init__(test_context, kafka, "process") + class StreamsSmokeTestShutdownDeadlockService(StreamsSmokeTestBaseService): - def __init__(self, context, kafka): - super(StreamsSmokeTestShutdownDeadlockService, self).__init__(context, kafka, "close-deadlock-test") + def __init__(self, test_context, kafka): + super(StreamsSmokeTestShutdownDeadlockService, self).__init__(test_context, kafka, "close-deadlock-test") diff --git a/tests/kafkatest/tests/streams/streams_shutdown_deadlock_test.py b/tests/kafkatest/tests/streams/streams_shutdown_deadlock_test.py index 5e4e7f28c683b..482da9c5d85f7 100644 --- a/tests/kafkatest/tests/streams/streams_shutdown_deadlock_test.py +++ b/tests/kafkatest/tests/streams/streams_shutdown_deadlock_test.py @@ -13,11 +13,10 @@ # See the License for the specific language governing permissions and # limitations under the License. -from ducktape.mark import ignore - from kafkatest.tests.kafka_test import KafkaTest from kafkatest.services.streams import StreamsSmokeTestShutdownDeadlockService + class StreamsShutdownDeadlockTest(KafkaTest): """ Simple test of Kafka Streams. From 866b33c16496ab654d59e69cb3e83587c52da673 Mon Sep 17 00:00:00 2001 From: "Colin P. Mccabe" Date: Wed, 18 Jan 2017 22:45:57 +0000 Subject: [PATCH 014/177] KAFKA-4457; Add BrokerApiVersionsCommand Author: Colin P. Mccabe Reviewers: Ewen Cheslack-Postava , Apurva Mehta , Ismael Juma Closes #2184 from cmccabe/KAFKA-4457 (cherry picked from commit 4a6f2c6cc0647a08f016a2d712a01ec02630cf87) Signed-off-by: Ismael Juma --- bin/kafka-broker-api-versions.sh | 17 ++++ .../apache/kafka/clients/NodeApiVersions.java | 34 +++++++- .../kafka/clients/NodeApiVersionsTest.java | 8 +- .../main/scala/kafka/admin/AdminClient.scala | 49 ++++++----- .../admin/BrokerApiVersionsCommand.scala | 81 +++++++++++++++++++ .../admin/BrokerApiVersionsCommandTest.scala | 61 ++++++++++++++ .../kafka/api/AdminClientTest.scala | 19 +++++ 7 files changed, 244 insertions(+), 25 deletions(-) create mode 100755 bin/kafka-broker-api-versions.sh create mode 100644 core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala create mode 100644 core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala diff --git a/bin/kafka-broker-api-versions.sh b/bin/kafka-broker-api-versions.sh new file mode 100755 index 0000000000000..4f560a0a60cd5 --- /dev/null +++ b/bin/kafka-broker-api-versions.sh @@ -0,0 +1,17 @@ +#!/bin/bash +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +exec $(dirname $0)/kafka-run-class.sh kafka.admin.BrokerApiVersionsCommand "$@" diff --git a/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java b/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java index 6acbb6318374c..b90009bd57dbe 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java +++ b/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.EnumMap; import java.util.Map; +import java.util.NoSuchElementException; import java.util.TreeMap; public class NodeApiVersions { @@ -56,10 +57,21 @@ public short usableVersion(ApiKeys apiKey) { } /** + * Convert the object to a string with no linebreaks.

    + * * This toString method is relatively expensive, so avoid calling it unless debug logging is turned on. */ @Override public String toString() { + return toString(false); + } + + /** + * Convert the object to a string. + * + * @param lineBreaks True if we should add a linebreak after each api. + */ + public String toString(boolean lineBreaks) { // The apiVersion collection may not be in sorted order. We put it into // a TreeMap before printing it out to ensure that we always print in // ascending order. @@ -73,11 +85,20 @@ public String toString() { if (!apiKeysText.containsKey(apiKey.id)) { StringBuilder bld = new StringBuilder(); bld.append(apiKey.name).append("("). - append(apiKey.id).append("): ").append("UNSUPPORTED"); + append(apiKey.id).append("): ").append("UNSUPPORTED"); apiKeysText.put(apiKey.id, bld.toString()); } } - return "{" + Utils.join(apiKeysText.values(), ", ") + "}"; + String separator = lineBreaks ? ",\n\t" : ", "; + StringBuilder bld = new StringBuilder(); + bld.append("("); + if (lineBreaks) + bld.append("\n\t"); + bld.append(Utils.join(apiKeysText.values(), separator)); + if (lineBreaks) + bld.append("\n"); + bld.append(")"); + return bld.toString(); } private String apiVersionToText(ApiVersion apiVersion) { @@ -106,4 +127,13 @@ private String apiVersionToText(ApiVersion apiVersion) { } return bld.toString(); } + + public ApiVersion apiVersion(ApiKeys apiKey) { + for (ApiVersion apiVersion : apiVersions) { + if (apiVersion.apiKey == apiKey.id) { + return apiVersion; + } + } + throw new NoSuchElementException(); + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java b/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java index b39a0aa88fabd..861a28f01f958 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java @@ -36,13 +36,13 @@ public void testUnsupportedVersionsToString() { NodeApiVersions versions = new NodeApiVersions( Collections.emptyList()); StringBuilder bld = new StringBuilder(); - String prefix = "{"; + String prefix = "("; for (ApiKeys apiKey : ApiKeys.values()) { bld.append(prefix).append(apiKey.name). append("(").append(apiKey.id).append("): UNSUPPORTED"); prefix = ", "; } - bld.append("}"); + bld.append(")"); assertEquals(bld.toString(), versions.toString()); } @@ -59,7 +59,7 @@ public void testVersionsToString() { } NodeApiVersions versions = new NodeApiVersions(versionList); StringBuilder bld = new StringBuilder(); - String prefix = "{"; + String prefix = "("; for (ApiKeys apiKey : ApiKeys.values()) { bld.append(prefix); if (apiKey == ApiKeys.CONTROLLED_SHUTDOWN_KEY) { @@ -80,7 +80,7 @@ public void testVersionsToString() { } prefix = ", "; } - bld.append("}"); + bld.append(")"); assertEquals(bld.toString(), versions.toString()); } diff --git a/core/src/main/scala/kafka/admin/AdminClient.scala b/core/src/main/scala/kafka/admin/AdminClient.scala index 33089d107a9ab..680c5e16f435b 100644 --- a/core/src/main/scala/kafka/admin/AdminClient.scala +++ b/core/src/main/scala/kafka/admin/AdminClient.scala @@ -16,6 +16,7 @@ import java.nio.ByteBuffer import java.util.{Collections, Properties} import java.util.concurrent.atomic.AtomicInteger +import org.apache.kafka.common.requests.ApiVersionsResponse.ApiVersion import kafka.common.KafkaException import kafka.coordinator.GroupOverview import kafka.utils.Logging @@ -28,11 +29,11 @@ import org.apache.kafka.common.network.Selector import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests.OffsetFetchResponse -import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{Cluster, Node, TopicPartition} import scala.collection.JavaConverters._ +import scala.util.Try class AdminClient(val time: Time, val requestTimeoutMs: Int, @@ -68,37 +69,42 @@ class AdminClient(val time: Time, def findCoordinator(groupId: String): Node = { val requestBuilder = new GroupCoordinatorRequest.Builder(groupId) val response = sendAnyNode(ApiKeys.GROUP_COORDINATOR, requestBuilder).asInstanceOf[GroupCoordinatorResponse] - Errors.forCode(response.errorCode()).maybeThrow() - response.node() + Errors.forCode(response.errorCode).maybeThrow() + response.node } def listGroups(node: Node): List[GroupOverview] = { val response = send(node, ApiKeys.LIST_GROUPS, new ListGroupsRequest.Builder()).asInstanceOf[ListGroupsResponse] - Errors.forCode(response.errorCode()).maybeThrow() - response.groups().asScala.map(group => GroupOverview(group.groupId(), group.protocolType())).toList + Errors.forCode(response.errorCode).maybeThrow() + response.groups.asScala.map(group => GroupOverview(group.groupId, group.protocolType)).toList + } + + def getApiVersions(node: Node): List[ApiVersion] = { + val response = send(node, ApiKeys.API_VERSIONS, new ApiVersionsRequest.Builder()).asInstanceOf[ApiVersionsResponse] + Errors.forCode(response.errorCode).maybeThrow() + response.apiVersions.asScala.toList } private def findAllBrokers(): List[Node] = { val request = MetadataRequest.Builder.allTopics() val response = sendAnyNode(ApiKeys.METADATA, request).asInstanceOf[MetadataResponse] - val errors = response.errors() + val errors = response.errors if (!errors.isEmpty) debug(s"Metadata request contained errors: $errors") - response.cluster().nodes().asScala.toList + response.cluster.nodes.asScala.toList } def listAllGroups(): Map[Node, List[GroupOverview]] = { - findAllBrokers.map { - case broker => - broker -> { - try { - listGroups(broker) - } catch { - case e: Exception => - debug(s"Failed to find groups from broker $broker", e) - List[GroupOverview]() - } + findAllBrokers.map { broker => + broker -> { + try { + listGroups(broker) + } catch { + case e: Exception => + debug(s"Failed to find groups from broker $broker", e) + List[GroupOverview]() } + } }.toMap } @@ -123,9 +129,14 @@ class AdminClient(val time: Time, if (response.hasError) throw response.error.exception response.maybeThrowFirstPartitionError - response.responseData().asScala.map { responseData => (responseData._1, responseData._2.offset) }.toMap + response.responseData.asScala.map { case (tp, partitionData) => (tp, partitionData.offset) }.toMap } + def listAllBrokerVersionInfo(): Map[Node, Try[NodeApiVersions]] = + findAllBrokers.map { broker => + broker -> Try[NodeApiVersions](new NodeApiVersions(getApiVersions(broker).asJava)) + }.toMap + /** * Case class used to represent a consumer of a consumer group */ @@ -252,6 +263,6 @@ object AdminClient { time, DefaultRequestTimeoutMs, highLevelClient, - bootstrapCluster.nodes().asScala.toList) + bootstrapCluster.nodes.asScala.toList) } } diff --git a/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala b/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala new file mode 100644 index 0000000000000..812bc9ddad77b --- /dev/null +++ b/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala @@ -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 kafka.admin + +import java.io.PrintStream +import java.util.Properties + +import kafka.utils.CommandLineUtils +import org.apache.kafka.common.utils.Utils +import org.apache.kafka.clients.CommonClientConfigs +import joptsimple._ + +import scala.util.{Failure, Success} + +/** + * A command for retrieving broker version information. + */ +object BrokerApiVersionsCommand { + + def main(args: Array[String]): Unit = { + execute(args, System.out) + } + + def execute(args: Array[String], out: PrintStream): Unit = { + val opts = new BrokerVersionCommandOptions(args) + val adminClient = createAdminClient(opts) + val brokerMap = adminClient.listAllBrokerVersionInfo() + brokerMap.foreach { case (broker, versionInfoOrError) => + versionInfoOrError match { + case Success(v) => out.print(s"${broker} -> ${v.toString(true)}\n") + case Failure(v) => out.print(s"${broker} -> ERROR: ${v}\n") + } + } + } + + private def createAdminClient(opts: BrokerVersionCommandOptions): AdminClient = { + val props = if (opts.options.has(opts.commandConfigOpt)) + Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt)) + else + new Properties() + props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt)) + AdminClient.create(props) + } + + class BrokerVersionCommandOptions(args: Array[String]) { + val BootstrapServerDoc = "REQUIRED: The server to connect to." + val CommandConfigDoc = "A property file containing configs to be passed to Admin Client." + + val parser = new OptionParser + val commandConfigOpt = parser.accepts("command-config", CommandConfigDoc) + .withRequiredArg + .describedAs("command config property file") + .ofType(classOf[String]) + val bootstrapServerOpt = parser.accepts("bootstrap-server", BootstrapServerDoc) + .withRequiredArg + .describedAs("server(s) to use for bootstrapping") + .ofType(classOf[String]) + val options = parser.parse(args : _*) + checkArgs() + + def checkArgs() { + // check required args + CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt) + } + } +} diff --git a/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala b/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala new file mode 100644 index 0000000000000..ff93f22c11f25 --- /dev/null +++ b/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.admin + +import java.io.{ByteArrayOutputStream, PrintStream} +import java.nio.charset.StandardCharsets + +import kafka.integration.KafkaServerTestHarness +import kafka.server.KafkaConfig +import kafka.utils.TestUtils +import org.apache.kafka.clients.NodeApiVersions +import org.apache.kafka.common.protocol.ApiKeys +import org.apache.kafka.common.requests.ApiVersionsResponse +import org.junit.Assert.{assertEquals, assertFalse, assertTrue} +import org.junit.Test + +class BrokerApiVersionsCommandTest extends KafkaServerTestHarness { + + def generateConfigs(): Seq[KafkaConfig] = TestUtils.createBrokerConfigs(1, zkConnect).map(KafkaConfig.fromProps) + + @Test + def checkBrokerApiVersionCommandOutput() { + val byteArrayOutputStream = new ByteArrayOutputStream + val printStream = new PrintStream(byteArrayOutputStream) + BrokerApiVersionsCommand.execute(Array("--bootstrap-server", brokerList), printStream) + val content = new String(byteArrayOutputStream.toByteArray, StandardCharsets.UTF_8) + val lineIter = content.split("\n").iterator + assertTrue(lineIter.hasNext) + assertEquals(s"$brokerList (id: 0 rack: null) -> (", lineIter.next) + val nodeApiVersions = new NodeApiVersions(ApiVersionsResponse.API_VERSIONS_RESPONSE.apiVersions) + for (apiKey <- ApiKeys.values) { + val apiVersion = nodeApiVersions.apiVersion(apiKey) + val versionRangeStr = + if (apiVersion.minVersion == apiVersion.maxVersion) apiVersion.minVersion.toString + else s"${apiVersion.minVersion} to ${apiVersion.maxVersion}" + val terminator = if (apiKey == ApiKeys.values.last) "" else "," + val usableVersion = nodeApiVersions.usableVersion(apiKey) + val line = s"\t${apiKey.name}(${apiKey.id}): $versionRangeStr [usable: $usableVersion]$terminator" + assertTrue(lineIter.hasNext) + assertEquals(line, lineIter.next) + } + assertTrue(lineIter.hasNext) + assertEquals(")", lineIter.next) + assertFalse(lineIter.hasNext) + } +} diff --git a/core/src/test/scala/integration/kafka/api/AdminClientTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientTest.scala index a62922caaf278..1e2749f6946ca 100644 --- a/core/src/test/scala/integration/kafka/api/AdminClientTest.scala +++ b/core/src/test/scala/integration/kafka/api/AdminClientTest.scala @@ -24,6 +24,7 @@ import kafka.utils.{Logging, TestUtils} import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.protocol.ApiKeys import org.junit.{Before, Test} import org.junit.Assert._ @@ -77,6 +78,24 @@ class AdminClientTest extends IntegrationTestHarness with Logging { assertEquals("consumer", group.protocolType) } + @Test + def testListAllBrokerVersionInfo() { + consumers.head.subscribe(Collections.singletonList(topic)) + TestUtils.waitUntilTrue(() => { + consumers.head.poll(0) + !consumers.head.assignment.isEmpty + }, "Expected non-empty assignment") + val brokerVersionInfos = client.listAllBrokerVersionInfo + val brokers = brokerList.split(",") + assertEquals(brokers.size, brokerVersionInfos.size) + for ((node, tryBrokerVersionInfo) <- brokerVersionInfos) { + val hostStr = s"${node.host}:${node.port}" + assertTrue(s"Unknown host:port pair $hostStr in brokerVersionInfos", brokers.contains(hostStr)) + val brokerVersionInfo = tryBrokerVersionInfo.get + assertEquals(0, brokerVersionInfo.usableVersion(ApiKeys.API_VERSIONS)) + } + } + @Test def testGetConsumerGroupSummary() { consumers.head.subscribe(Collections.singletonList(topic)) From 5562d63e8a02fdddaabe066437fe7674d28d5d01 Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Wed, 18 Jan 2017 15:53:17 -0800 Subject: [PATCH 015/177] KAFKA-4060; Follow-up: Throw exceptions when internal topics to create already exist with unexpected number of partitions Re-branched the trunk and applied the changes to the new branch to simplify commit log. Author: Hojjat Jafarpour Reviewers: Ismael Juma, Damian Guy, Eno Thereska, Guozhang Wang Closes #2389 from hjafarpour/KAFKA-4060-Remove-ZkClient-dependency-in-Kafka-Streams-followup-from-trunk Address Ismael's comments upon merging (cherry picked from commit 8e2cbae8b12cd71f99f13953b744163f68021323) Signed-off-by: Guozhang Wang --- .../internals/InternalTopicManager.java | 28 ++-- .../internals/StreamsKafkaClient.java | 104 +++++---------- .../internals/InternalTopicManagerTest.java | 124 ++++++++++++++++++ .../StreamPartitionAssignorTest.java | 25 ---- 4 files changed, 174 insertions(+), 107 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index 8bb5a3df65314..133375fd17c71 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -48,9 +48,13 @@ public InternalTopicManager(final StreamsKafkaClient streamsKafkaClient, final i } /** - * Prepares the set of given internal topics. If the topic with the correct number of partitions exists ignores it. For the ones with different number of - * partitions delete them and create new ones with correct number of partitons along with the non existing topics. + * Prepares a given internal topic. + * If the topic does not exist creates a new topic. + * If the topic with the correct number of partitions exists ignores it. + * If the topic exists already but has different number of partitions we fail and throw exception requesting user to reset the app before restarting again. + * * @param topic + * @param numPartitions */ public void makeReady(final InternalTopicConfig topic, int numPartitions) { @@ -58,11 +62,9 @@ public void makeReady(final InternalTopicConfig topic, int numPartitions) { topics.put(topic, numPartitions); for (int i = 0; i < MAX_TOPIC_READY_TRY; i++) { try { - Collection topicMetadatas = streamsKafkaClient.fetchTopicMetadata(); - Map topicsToBeDeleted = getTopicsToBeDeleted(topics, topicMetadatas); - Map topicsToBeCreated = filterExistingTopics(topics, topicMetadatas); - topicsToBeCreated.putAll(topicsToBeDeleted); - streamsKafkaClient.deleteTopics(topicsToBeDeleted); + Collection topicsMetadata = streamsKafkaClient.fetchTopicsMetadata(); + validateTopicPartitons(topics, topicsMetadata); + Map topicsToBeCreated = filterExistingTopics(topics, topicsMetadata); streamsKafkaClient.createTopics(topicsToBeCreated, replicationFactor, windowChangeLogAdditionalRetention); return; } catch (StreamsException ex) { @@ -99,24 +101,22 @@ private Map filterExistingTopics(final Map getTopicsToBeDeleted(final Map topicsPartitionsMap, Collection topicsMetadata) { + private void validateTopicPartitons(final Map topicsPartitionsMap, Collection topicsMetadata) { Map existingTopicNamesPartitions = getExistingTopicNamesPartitions(topicsMetadata); - Map deleteTopics = new HashMap<>(); - // Add the topics that don't exist to the nonExistingTopics. for (InternalTopicConfig topic: topicsPartitionsMap.keySet()) { if (existingTopicNamesPartitions.get(topic.name()) != null) { if (existingTopicNamesPartitions.get(topic.name()) != topicsPartitionsMap.get(topic)) { - deleteTopics.put(topic, topicsPartitionsMap.get(topic)); + throw new StreamsException("Internal topic with invalid partitons. Use 'kafka.tools.StreamsResetter' tool to clean up invalid topics before processing."); } } } - return deleteTopics; } private Map getExistingTopicNamesPartitions(Collection topicsMetadata) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java index f33a4e40f6430..8ea570faa5379 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -33,11 +33,8 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.CreateTopicsRequest; import org.apache.kafka.common.requests.CreateTopicsResponse; -import org.apache.kafka.common.requests.DeleteTopicsRequest; -import org.apache.kafka.common.requests.DeleteTopicsResponse; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; -import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.StreamsConfig; @@ -45,14 +42,13 @@ import java.io.IOException; import java.net.InetSocketAddress; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; -import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.LinkedHashMap; import java.util.Properties; +import java.util.HashMap; import java.util.Collection; +import java.util.Arrays; import java.util.concurrent.TimeUnit; public class StreamsKafkaClient { @@ -125,10 +121,13 @@ public void createTopics(final Map topicsMap, fina topicRequestDetails.put(internalTopicConfig.name(), topicDetails); } - final CreateTopicsRequest.Builder createTopicsRequest = - new CreateTopicsRequest.Builder(topicRequestDetails, - streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); - final ClientResponse clientResponse = sendRequest(createTopicsRequest); + + final ClientRequest clientRequest = kafkaClient.newClientRequest(getBrokerId(), new CreateTopicsRequest.Builder(topicRequestDetails, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)), Time.SYSTEM.milliseconds(), true, null); + final ClientResponse clientResponse = sendRequest(clientRequest); + + if (!clientResponse.hasResponse()) { + throw new StreamsException("Empty response for client request."); + } if (!(clientResponse.responseBody() instanceof CreateTopicsResponse)) { throw new StreamsException("Inconsistent response type for internal topic creation request. Expected CreateTopicsResponse but received " + clientResponse.responseBody().getClass().getName()); } @@ -146,51 +145,8 @@ public void createTopics(final Map topicsMap, fina } } - /** - * Delets a set of topics. - * - * @param topics - */ - public void deleteTopics(final Map topics) { - - final Set topicNames = new HashSet<>(); - for (InternalTopicConfig internalTopicConfig: topics.keySet()) { - topicNames.add(internalTopicConfig.name()); - } - deleteTopics(topicNames); - } - - /** - * Delete a set of topics in one request. - * - * @param topics - */ - private void deleteTopics(final Set topics) { - - final DeleteTopicsRequest.Builder deleteTopicsRequest = - new DeleteTopicsRequest.Builder(topics, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); - final ClientResponse clientResponse = sendRequest(deleteTopicsRequest); - if (!(clientResponse.responseBody() instanceof DeleteTopicsResponse)) { - throw new StreamsException("Inconsistent response type for internal topic deletion request. Expected DeleteTopicsResponse but received " + clientResponse.responseBody().getClass().getName()); - } - final DeleteTopicsResponse deleteTopicsResponse = (DeleteTopicsResponse) clientResponse.responseBody(); - for (Map.Entry entry : deleteTopicsResponse.errors().entrySet()) { - if (entry.getValue() != Errors.NONE) { - throw new StreamsException("Could not delete topic: " + entry.getKey() + " due to " + entry.getValue().message()); - } - } - - } - - /** - * Send a request to kafka broker of this client. Keep polling until the corresponding response is received. - * - * @param request - */ - private ClientResponse sendRequest(final AbstractRequest.Builder request) { - + private String getBrokerId() { String brokerId = null; - final Metadata metadata = new Metadata(streamsConfig.getLong(StreamsConfig.RETRY_BACKOFF_MS_CONFIG), streamsConfig.getLong(StreamsConfig.METADATA_MAX_AGE_CONFIG)); final List addresses = ClientUtils.parseAndValidateAddresses(streamsConfig.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); metadata.update(Cluster.bootstrap(addresses), Time.SYSTEM.milliseconds()); @@ -211,12 +167,11 @@ private ClientResponse sendRequest(final AbstractRequest.Builder request) { if (brokerId == null) { throw new StreamsException("Could not find any available broker."); } + return brokerId; + } - final ClientRequest clientRequest = kafkaClient.newClientRequest( - brokerId, request, Time.SYSTEM.milliseconds(), true, null); - + private ClientResponse sendRequest(final ClientRequest clientRequest) { kafkaClient.send(clientRequest, Time.SYSTEM.milliseconds()); - final long responseTimeout = Time.SYSTEM.milliseconds() + streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG); // Poll for the response. while (Time.SYSTEM.milliseconds() < responseTimeout) { @@ -229,24 +184,27 @@ private ClientResponse sendRequest(final AbstractRequest.Builder request) { if (response.requestHeader().correlationId() == clientRequest.correlationId()) { return response; } else { - throw new StreamsException("Inconsistent response received from broker " + brokerId + - ", expected correlation id " + clientRequest.correlationId() + ", but received " + + throw new StreamsException("Inconsistent response received from the broker " + clientRequest.destination() + ", expected correlation id " + clientRequest.correlationId() + ", but received " + response.requestHeader().correlationId()); } } } throw new StreamsException("Failed to get response from broker within timeout"); + } - /** - * Get the metadata for a topic. + /** + * Fetch the metadata for a topic. * @param topic * @return */ - public MetadataResponse.TopicMetadata getTopicMetadata(final String topic) { - - final ClientResponse clientResponse = sendRequest(MetadataRequest.Builder.allTopics()); + public MetadataResponse.TopicMetadata fetchTopicMetadata(final String topic) { + final ClientRequest clientRequest = kafkaClient.newClientRequest(getBrokerId(), new MetadataRequest.Builder(Arrays.asList(topic)), Time.SYSTEM.milliseconds(), true, null); + final ClientResponse clientResponse = sendRequest(clientRequest); + if (!clientResponse.hasResponse()) { + throw new StreamsException("Empty response for client request."); + } if (!(clientResponse.responseBody() instanceof MetadataResponse)) { throw new StreamsException("Inconsistent response type for internal topic metadata request. Expected MetadataResponse but received " + clientResponse.responseBody().getClass().getName()); } @@ -260,8 +218,18 @@ public MetadataResponse.TopicMetadata getTopicMetadata(final String topic) { } - public Collection fetchTopicMetadata() { - final ClientResponse clientResponse = sendRequest(MetadataRequest.Builder.allTopics()); + /** + * Fetch the metadata for all topics + * + * @return + */ + public Collection fetchTopicsMetadata() { + + final ClientRequest clientRequest = kafkaClient.newClientRequest(getBrokerId(), new MetadataRequest.Builder(null), Time.SYSTEM.milliseconds(), true, null); + final ClientResponse clientResponse = sendRequest(clientRequest); + if (!clientResponse.hasResponse()) { + throw new StreamsException("Empty response for client request."); + } if (!(clientResponse.responseBody() instanceof MetadataResponse)) { throw new StreamsException("Inconsistent response type for internal topic metadata request. Expected MetadataResponse but received " + clientResponse.responseBody().getClass().getName()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java new file mode 100644 index 0000000000000..f82809917e5a5 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java @@ -0,0 +1,124 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

    + * http://www.apache.org/licenses/LICENSE-2.0 + *

    + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.common.Node; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.test.MockTimestampExtractor; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.Properties; +import java.util.Arrays; +import java.util.ArrayList; + +import static org.apache.kafka.streams.processor.internals.InternalTopicManager.WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT; + +public class InternalTopicManagerTest { + + private String userEndPoint = "localhost:2171"; + StreamsConfig config; + MockStreamKafkaClient streamsKafkaClient; + + @Before + public void init() { + config = new StreamsConfig(configProps()); + streamsKafkaClient = new MockStreamKafkaClient(config); + } + + @Test + public void shouldCreateRequiredTopics() throws Exception { + + streamsKafkaClient.setReturnCorrectTopic(true); + InternalTopicManager internalTopicManager = new InternalTopicManager(streamsKafkaClient, 1, WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT); + internalTopicManager.makeReady(new InternalTopicConfig("test_topic", Collections.singleton(InternalTopicConfig.CleanupPolicy.compact), null), 1); + } + + @Test + public void shouldNotCreateTopicIfExistsWithDifferentPartitions() throws Exception { + + streamsKafkaClient.setReturnCorrectTopic(true); + InternalTopicManager internalTopicManager = new InternalTopicManager(streamsKafkaClient, 1, WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT); + boolean exceptionWasThrown = false; + try { + internalTopicManager.makeReady(new InternalTopicConfig("test_topic", Collections.singleton(InternalTopicConfig.CleanupPolicy.compact), null), 2); + } catch (StreamsException e) { + exceptionWasThrown = true; + } + Assert.assertTrue(exceptionWasThrown); + } + + private Properties configProps() { + return new Properties() { + { + setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "Internal-Topic-ManagerTest"); + setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, userEndPoint); + setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); + setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()); + } + }; + } + + private class MockStreamKafkaClient extends StreamsKafkaClient { + public MockStreamKafkaClient(final StreamsConfig streamsConfig) { + super(streamsConfig); + } + + public boolean isReturnCorrectTopic() { + return returnCorrectTopic; + } + + public void setReturnCorrectTopic(boolean returnCorrectTopic) { + this.returnCorrectTopic = returnCorrectTopic; + } + + boolean returnCorrectTopic = false; + + + @Override + public void createTopics(final Map topicsMap, final int replicationFactor, final long windowChangeLogAdditionalRetention) { + + } + + @Override + public MetadataResponse.TopicMetadata fetchTopicMetadata(final String topic) { + + if (returnCorrectTopic) { + MetadataResponse.PartitionMetadata partitionMetadata = new MetadataResponse.PartitionMetadata(Errors.NONE, 1, null, new ArrayList(), new ArrayList()); + MetadataResponse.TopicMetadata topicMetadata = new MetadataResponse.TopicMetadata(Errors.NONE, topic, true, Arrays.asList(partitionMetadata)); + return topicMetadata; + } + return null; + } + + @Override + public Collection fetchTopicsMetadata() { + if (returnCorrectTopic) { + return Arrays.asList(fetchTopicMetadata("test_topic")); + } + return null; + } + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java index 5aa40c82733b5..c212f1412d6d7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.clients.consumer.MockConsumer; import org.apache.kafka.clients.consumer.internals.PartitionAssignor; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; @@ -1062,28 +1061,4 @@ private AssignmentInfo checkAssignment(Set expectedTopics, PartitionAssi return info; } - private class MockInternalTopicManager extends InternalTopicManager { - - Map readyTopics = new HashMap<>(); - MockConsumer restoreConsumer; - - MockInternalTopicManager(StreamsConfig streamsConfig, MockConsumer restoreConsumer) { - super(new StreamsKafkaClient(streamsConfig), 0, 0); - - this.restoreConsumer = restoreConsumer; - } - - @Override - public void makeReady(InternalTopicConfig topic, int numPartitions) { - readyTopics.put(topic.name(), numPartitions); - - List partitions = new ArrayList<>(); - for (int i = 0; i < numPartitions; i++) { - partitions.add(new PartitionInfo(topic.name(), i, null, null, null)); - } - - restoreConsumer.updatePartitions(topic.name(), partitions); - } - } - } From ae5591a3af45832d9a50e2aaa31b11c039d8d389 Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Thu, 19 Jan 2017 08:35:39 -0800 Subject: [PATCH 016/177] KAFKA-4589; SASL/SCRAM documentation Author: Rajini Sivaram Reviewers: Ismael Juma , Gwen Shapira , Sriharsha Chintalapani , Jun Rao Closes #2369 from rajinisivaram/KAFKA-4589 (cherry picked from commit 666abafcc54f8cab64912355dba4c8ada8e44827) Signed-off-by: Jun Rao --- docs/security.html | 317 +++++++++++++++++++++++++++++++-------------- 1 file changed, 218 insertions(+), 99 deletions(-) diff --git a/docs/security.html b/docs/security.html index 8cb867eb04cee..81d5c403c134e 100644 --- a/docs/security.html +++ b/docs/security.html @@ -19,8 +19,12 @@

    7.1 Security Overview

    In release 0.9.0.0, the Kafka community added a number of features that, used either separately or together, increases security in a Kafka cluster. The following security measures are currently supported:
      -
    1. Authentication of connections to brokers from clients (producers and consumers), other brokers and tools, using either SSL or SASL (Kerberos). - SASL/PLAIN can also be used from release 0.10.0.0 onwards.
    2. +
    3. Authentication of connections to brokers from clients (producers and consumers), other brokers and tools, using either SSL or SASL. Kafka supports the following SASL mechanisms: +
        +
      • SASL/GSSAPI (Kerberos) - starting at version 0.9.0.0
      • +
      • SASL/PLAIN - starting at version 0.10.0.0
      • +
      • SASL/SCRAM-SHA-256 and SASL/SCRAM-SHA-512 - starting at version 0.10.2.0
      • +
    4. Authentication of connections from brokers to ZooKeeper
    5. Encryption of data transferred between brokers and clients, between brokers, or between brokers and tools using SSL (Note that there is a performance degradation when SSL is enabled, the magnitude of which depends on the CPU type and the JVM implementation.)
    6. Authorization of read / write operations by clients
    7. @@ -211,117 +215,125 @@

      7.2 Encryption and Authentication

      7.3 Authentication using SASL

        -
      1. SASL configuration for Kafka brokers

        -
          -
        1. Select one or more supported mechanisms to enable in the broker. GSSAPI - and PLAIN are the mechanisms currently supported in Kafka.
        2. -
        3. Add a JAAS config file for the selected mechanisms as described in the examples - for setting up GSSAPI (Kerberos) - or PLAIN.
        4. -
        5. Pass the JAAS config file location as JVM parameter to each Kafka broker. - For example: -
              -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
        6. -
        7. Configure a SASL port in server.properties, by adding at least one of - SASL_PLAINTEXT or SASL_SSL to the listeners parameter, which - contains one or more comma-separated values: -
              listeners=SASL_PLAINTEXT://host.name:port
          - If SASL_SSL is used, then SSL must also be - configured. If you are only configuring a SASL port (or if you want - the Kafka brokers to authenticate each other using SASL) then make sure - you set the same SASL protocol for inter-broker communication: -
              security.inter.broker.protocol=SASL_PLAINTEXT (or SASL_SSL)
        8. -
        9. Enable one or more SASL mechanisms in server.properties: -
              sasl.enabled.mechanisms=GSSAPI (,PLAIN)
        10. -
        11. Configure the SASL mechanism for inter-broker communication in server.properties - if using SASL for inter-broker communication: -
              sasl.mechanism.inter.broker.protocol=GSSAPI (or PLAIN)
        12. -
        13. Follow the steps in GSSAPI (Kerberos) - or PLAIN to configure SASL - for the enabled mechanisms. To enable multiple mechanisms in the broker, follow - the steps here.
        14. - Important notes: +
        15. JAAS configuration

          +

          Kafka uses the Java Authentication and Authorization Service + (JAAS) + for SASL configuration.

            -
          1. KafkaServer is the section name in the JAAS file used by each +
          2. JAAS configuration for Kafka brokers
            + +

            KafkaServer is the section name in the JAAS file used by each KafkaServer/Broker. This section provides SASL configuration options for the broker including any SASL client connections made by the broker - for inter-broker communication.

          3. -
          4. Client section is used to authenticate a SASL connection with + for inter-broker communication.

            + +

            Client section is used to authenticate a SASL connection with zookeeper. It also allows the brokers to set SASL ACL on zookeeper nodes which locks these nodes down so that only the brokers can modify it. It is necessary to have the same principal name across all brokers. If you want to use a section name other than Client, set the system property zookeeper.sasl.client to the appropriate - name (e.g., -Dzookeeper.sasl.client=ZkClient).

          5. -
          6. ZooKeeper uses "zookeeper" as the service name by default. If you + name (e.g., -Dzookeeper.sasl.client=ZkClient).

            + +

            ZooKeeper uses "zookeeper" as the service name by default. If you want to change this, set the system property zookeeper.sasl.client.username to the appropriate name - (e.g., -Dzookeeper.sasl.client.username=zk).

          7. -
          -
        -
      2. -
      3. SASL configuration for Kafka clients

        - SASL authentication is only supported for the new Java Kafka producer and - consumer, the older API is not supported. JAAS configuration for clients may - be specified as a static JAAS config file or using the client configuration property - sasl.jaas.config. - To configure SASL authentication on the clients: -
          -
        1. Select a SASL mechanism for authentication.
        2. -
        3. Configure the following properties in producer.properties or - consumer.properties: -
              security.protocol=SASL_PLAINTEXT (or SASL_SSL)
          -    sasl.mechanism=GSSAPI (or PLAIN)
        4. -
        5. Follow the steps in GSSAPI (Kerberos) - or PLAIN to configure SASL - for the selected mechanism.
        6. -
        7. Configure JAAS using client configuration property - or static JAAS config file as described below.
        8. -
        -
          -
        1. JAAS configuration using client configuration property
          -

          Clients may specify JAAS configuration as a producer or consumer property without - creating a physical configuration file. This mode also enables different producers - and consumers within the same JVM to use different credentials by specifying - different properties for each client. If both static JAAS configuration system property - java.security.auth.login.config and client property sasl.jaas.config - are specified, the client property will be used.

          - - To configure SASL authentication on the clients using configuration property: -
            -
          1. Configure the property sasl.jaas.config in producer.properties or - consumer.properties to be the JAAS login module section of the selected mechanism. - For example, PLAIN - credentials may be configured as: -
                sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule required username="alice" password="alice-secret";
          2. - See GSSAPI (Kerberos) or PLAIN - for full example configurations. -
          -
        2. -
        3. JAAS configuration using static config file
          - To configure SASL authentication on the clients using static JAAS config file: + (e.g., -Dzookeeper.sasl.client.username=zk).

        4. + +
        5. JAAS configuration for Kafka clients
          + +

          Clients may configure JAAS using the client configuration property + sasl.jaas.config + or using the static JAAS config file + similar to brokers.

          +
            -
          1. Add a JAAS config file with a client login section named KafkaClient. Configure - a login module in KafkaClient for the selected mechanism as described in the examples - for setting up GSSAPI (Kerberos) - or PLAIN. - For example, GSSAPI - credentials may be configured as: -
            -    KafkaClient {
            +            
          2. JAAS configuration using client configuration property
            +

            Clients may specify JAAS configuration as a producer or consumer property without + creating a physical configuration file. This mode also enables different producers + and consumers within the same JVM to use different credentials by specifying + different properties for each client. If both static JAAS configuration system property + java.security.auth.login.config and client property sasl.jaas.config + are specified, the client property will be used.

            + +

            See GSSAPI (Kerberos), + PLAIN or + SCRAM for example configurations.

          3. + +
          4. JAAS configuration using static config file
            + To configure SASL authentication on the clients using static JAAS config file: +
              +
            1. Add a JAAS config file with a client login section named KafkaClient. Configure + a login module in KafkaClient for the selected mechanism as described in the examples + for setting up GSSAPI (Kerberos), + PLAIN or + SCRAM. + For example, GSSAPI + credentials may be configured as: +
              +        KafkaClient {
                       com.sun.security.auth.module.Krb5LoginModule required
                       useKeyTab=true
                       storeKey=true
                       keyTab="/etc/security/keytabs/kafka_client.keytab"
                       principal="kafka-client-1@EXAMPLE.COM";
                   };
              - See GSSAPI (Kerberos) or PLAIN - for example configurations of each mechanism.
            2. -
            3. Pass the JAAS config file location as JVM parameter to each client JVM. For example: -
                  -Djava.security.auth.login.config=/etc/kafka/kafka_client_jaas.conf
            4. + +
            5. Pass the JAAS config file location as JVM parameter to each client JVM. For example: +
                  -Djava.security.auth.login.config=/etc/kafka/kafka_client_jaas.conf
            6. +
            +
          5. +
          +
        6. +
        +
      4. +
      5. SASL configuration

        + +

        SASL may be used with PLAINTEXT or SSL as the transport layer using the + security protocol SASL_PLAINTEXT or SASL_SSL respectively. If SASL_SSL is + used, then SSL must also be configured.

        + +
          +
        1. SASL mechanisms
          + Kafka supports the following SASL mechanisms: + +
        2. +
        3. SASL configuration for Kafka brokers
          +
            +
          1. Configure a SASL port in server.properties, by adding at least one of + SASL_PLAINTEXT or SASL_SSL to the listeners parameter, which + contains one or more comma-separated values: +
                listeners=SASL_PLAINTEXT://host.name:port
            + If you are only configuring a SASL port (or if you want + the Kafka brokers to authenticate each other using SASL) then make sure + you set the same SASL protocol for inter-broker communication: +
                security.inter.broker.protocol=SASL_PLAINTEXT (or SASL_SSL)
          2. +
          3. Select one or more supported mechanisms + to enable in the broker and follow the steps to configure SASL for the mechanism. + To enable multiple mechanisms in the broker, follow the steps + here.
        4. +
        5. SASL configuration for Kafka clients
          +

          SASL authentication is only supported for the new Java Kafka producer and + consumer, the older API is not supported.

          + +

          To configure SASL authentication on the clients, select a SASL + mechanism that is enabled in + the broker for client authentication and follow the steps to configure SASL + for the selected mechanism.

      6. Authentication using SASL/Kerberos

        @@ -502,6 +514,111 @@

        7.3 Authentication using SASL

      + +
    8. Authentication using SASL/SCRAM

      +

      Salted Challenge Response Authentication Mechanism (SCRAM) is a family of SASL mechanisms that + addresses the security concerns with traditional mechanisms that perform username/password authentication + like PLAIN and DIGEST-MD5. The mechanism is defined in RFC 5802. + Kafka supports SCRAM-SHA-256 and SCRAM-SHA-512 which + can be used with TLS to perform secure authentication. The username is used as the authenticated + Principal for configuration of ACLs etc. The default SCRAM implementation in Kafka + stores SCRAM credentials in Zookeeper and is suitable for use in Kafka installations where Zookeeper + is on a private network. Refer to Security Considerations + for more details.

      +
        +
      1. Creating SCRAM Credentials
        +

        The SCRAM implementation in Kafka uses Zookeeper as credential store. Credentials can be created in + Zookeeper using kafka-configs.sh. For each SCRAM mechanism enabled, credentials must be created + by adding a config with the mechanism name. Credentials for inter-broker communication must be created + before Kafka brokers are started. Client credentials may be created and updated dynamically and updated + credentials will be used to authenticate new connections.

        +

        Create SCRAM credentials for user alice with password alice-secret: +

        +    bin/kafka-configs.sh --zookeeper localhost:2181 --alter --add-config 'SCRAM-SHA-256=[iterations=8192,password=alice-secret],SCRAM-SHA-512=[password=alice-secret]' --entity-type users --entity-name alice
        +        
        +

        The default iteration count of 4096 is used if iterations are not specified. A random salt is created + and the SCRAM identity consisting of salt, iterations, StoredKey and ServerKey are stored in Zookeeper. + See RFC 5802 for details on SCRAM identity and the individual fields. +

        The following examples also require a user admin for inter-broker communication which can be created using: +

        +    bin/kafka-configs.sh --zookeeper localhost:2181 --alter --add-config 'SCRAM-SHA-256=[password=admin-secret],SCRAM-SHA-512=[password=admin-secret]' --entity-type users --entity-name admin
        +        
        +

        Existing credentials may be listed using the --describe option: +

        +   bin/kafka-configs.sh --zookeeper localhost:2181 --describe --entity-type users --entity-name alice
        +        
        +

        Credentials may be deleted for one or more SCRAM mechanisms using the --delete option: +

        +   bin/kafka-configs.sh --zookeeper localhost:2181 --alter --delete-config 'SCRAM-SHA-512' --entity-type users --entity-name alice
        +        
        +
      2. +
      3. Configuring Kafka Brokers
        +
          +
        1. Add a suitably modified JAAS file similar to the one below to each Kafka broker's config directory, let's call it kafka_server_jaas.conf for this example: +
          +    KafkaServer {
          +        org.apache.kafka.common.security.scram.ScramLoginModule required
          +        username="admin"
          +        password="admin-secret"
          +    };
          + The properties username and password in the KafkaServer section are used by + the broker to initiate connections to other brokers. In this example, admin is the user for + inter-broker communication.
        2. +
        3. Pass the JAAS config file location as JVM parameter to each Kafka broker: +
              -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
        4. +
        5. Configure SASL port and SASL mechanisms in server.properties as described here. For example: +
          +    listeners=SASL_SSL://host.name:port
          +    security.inter.broker.protocol=SASL_SSL
          +    sasl.mechanism.inter.broker.protocol=SCRAM-SHA-256 (or SCRAM-SHA-512)
          +    sasl.enabled.mechanisms=SCRAM-SHA-256 (or SCRAM-SHA-512)
        6. +
        +
      4. + +
      5. Configuring Kafka Clients
        + To configure SASL authentication on the clients: +
          +
        1. Configure the JAAS configuration property for each client in producer.properties or consumer.properties. + The login module describes how the clients like producer and consumer can connect to the Kafka Broker. + The following is an example configuration for a client for the SCRAM mechanisms: +
          +   sasl.jaas.config=org.apache.kafka.common.security.scram.ScramLoginModule required \
          +        username="alice" \
          +        password="alice-secret";
          + +

          The options username and password are used by clients to configure + the user for client connections. In this example, clients connect to the broker as user alice. + Different clients within a JVM may connect as different users by specifying different user names + and passwords in sasl.jaas.config.

          + +

          JAAS configuration for clients may alternatively be specified as a JVM parameter similar to brokers + as described here. Clients use the login section named + KafkaClient. This option allows only one user for all client connections from a JVM.

        2. + +
        3. Configure the following properties in producer.properties or consumer.properties: +
          +    security.protocol=SASL_SSL
          +    sasl.mechanism=SCRAM-SHA-256 (or SCRAM-SHA-512)
        4. +
        +
      6. +
      7. Security Considerations for SASL/SCRAM
        +
          +
        • The default implementation of SASL/SCRAM in Kafka stores SCRAM credentials in Zookeeper. This + is suitable for production use in installations where Zookeeper is secure and on a private network.
        • +
        • Kafka supports only the strong hash functions SHA-256 and SHA-512 with a minimum iteration count + of 4096. Strong hash functions combined with strong passwords and high iteration counts protect + against brute force attacks if Zookeeper security is compromised.
        • +
        • SCRAM should be used only with TLS-encryption to prevent interception of SCRAM exchanges. This + protects against dictionary or brute force attacks and against impersonation if Zookeeper is compromised.
        • +
        • The default SASL/SCRAM implementation may be overridden using custom login modules in installations + where Zookeeper is not secure. See here for details.
        • +
        • For more details on security considerations, refer to + RFC 5802. +
        +
      8. +
      +
    9. +
    10. Enabling multiple SASL mechanisms in a broker

      1. Specify configuration for the login modules of all enabled mechanisms in the KafkaServer section of the JAAS config file. For example: @@ -519,12 +636,14 @@

        7.3 Authentication using SASL

      2. -
      3. Enable the SASL mechanisms in server.properties:
            sasl.enabled.mechanisms=GSSAPI,PLAIN
      4. +
      5. Enable the SASL mechanisms in server.properties:
            sasl.enabled.mechanisms=GSSAPI,PLAIN,SCRAM-SHA-256,SCRAM-SHA-512
      6. Specify the SASL security protocol and mechanism for inter-broker communication in server.properties if required: -
            security.inter.broker.protocol=SASL_PLAINTEXT (or SASL_SSL)
        -        sasl.mechanism.inter.broker.protocol=GSSAPI (or PLAIN)
      7. -
      8. Follow the mechanism-specific steps in GSSAPI (Kerberos) - and PLAIN to configure SASL for the enabled mechanisms.
      9. +
        +    security.inter.broker.protocol=SASL_PLAINTEXT (or SASL_SSL)
        +    sasl.mechanism.inter.broker.protocol=GSSAPI (or one of the other enabled mechanisms)
        +
      10. Follow the mechanism-specific steps in GSSAPI (Kerberos), + PLAIN and SCRAM + to configure SASL for the enabled mechanisms.
    11. Modifying SASL mechanism in a Running Cluster

      From 770bab39916e213443aff8f696c6097b551b9408 Mon Sep 17 00:00:00 2001 From: Yuto Kawamura Date: Thu, 19 Jan 2017 09:47:34 -0800 Subject: [PATCH 017/177] KAFKA-4614; Forcefully unmap mmap of OffsetIndex to prevent long GC pause Issue: https://issues.apache.org/jira/browse/KAFKA-4614 Fixes the problem that the broker threads suffered by long GC pause. When GC thread collects mmap objects which were created for index files, it unmaps memory mapping so kernel turns to delete a file physically. This work may transparently read file's metadata from physical disk if it's not available on cache. This seems to happen typically when we're using G1GC, due to it's strategy to left a garbage for a long time if other objects in the same region are still alive. See the link for the details. Author: Yuto Kawamura Reviewers: Apurva Mehta , Guozhang Wang , Ismael Juma , Closes #2352 from kawamuray/KAFKA-4614-force-munmap-for-index (cherry picked from commit 5fc530bc483db145e0cba3b63a57d6d6a7c547f2) Signed-off-by: Jun Rao --- .../src/main/scala/kafka/log/AbstractIndex.scala | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/log/AbstractIndex.scala b/core/src/main/scala/kafka/log/AbstractIndex.scala index 77ef0f73f050e..6f850d54ecf75 100644 --- a/core/src/main/scala/kafka/log/AbstractIndex.scala +++ b/core/src/main/scala/kafka/log/AbstractIndex.scala @@ -17,7 +17,7 @@ package kafka.log -import java.io.{File, RandomAccessFile} +import java.io.{File, IOException, RandomAccessFile} import java.nio.{ByteBuffer, MappedByteBuffer} import java.nio.channels.FileChannel import java.util.concurrent.locks.{Lock, ReentrantLock} @@ -141,8 +141,16 @@ abstract class AbstractIndex[K, V](@volatile var file: File, val baseOffset: Lon */ def delete(): Boolean = { info(s"Deleting index ${file.getAbsolutePath}") - if(Os.isWindows) + inLock(lock) { + // On JVM, a memory mapping is typically unmapped by garbage collector. + // However, in some cases it can pause application threads(STW) for a long moment reading metadata from a physical disk. + // To prevent this, we forcefully cleanup memory mapping within proper execution which never affects API responsiveness. + // See https://issues.apache.org/jira/browse/KAFKA-4614 for the details. CoreUtils.swallow(forceUnmap(mmap)) + // Accessing unmapped mmap crashes JVM by SEGV. + // Accessing it after this method called sounds like a bug but for safety, assign null and do not allow later access. + mmap = null + } file.delete() } @@ -185,7 +193,7 @@ abstract class AbstractIndex[K, V](@volatile var file: File, val baseOffset: Lon def truncateTo(offset: Long): Unit /** - * Forcefully free the buffer's mmap. We do this only on windows. + * Forcefully free the buffer's mmap. */ protected def forceUnmap(m: MappedByteBuffer) { try { @@ -279,4 +287,4 @@ abstract class AbstractIndex[K, V](@volatile var file: File, val baseOffset: Lon object IndexSearchType extends Enumeration { type IndexSearchEntity = Value val KEY, VALUE = Value -} \ No newline at end of file +} From 17dd4db29fe7f930ca1c5d0fb5687a48c364f2d7 Mon Sep 17 00:00:00 2001 From: Vahid Hashemian Date: Thu, 19 Jan 2017 16:35:12 -0800 Subject: [PATCH 018/177] KAFKA-4547; Avoid incorrect reset of paused partitions to the committed offsets Author: Vahid Hashemian Reviewers: Jason Gustafson Closes #2341 from vahidhashemian/KAFKA-4547 --- .../kafka/clients/consumer/KafkaConsumer.java | 6 +- .../clients/consumer/internals/Fetcher.java | 2 +- .../consumer/internals/SubscriptionState.java | 14 ++++- .../clients/consumer/KafkaConsumerTest.java | 55 ++++++++++++++++ .../consumer/internals/FetcherTest.java | 62 +++++++++++++++++++ 5 files changed, 132 insertions(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 1b033e9e06639..be212db8a4a1f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -1584,9 +1584,9 @@ private void updateFetchPositions(Set partitions) { // the user is manually assigning partitions and managing their own offsets). fetcher.resetOffsetsIfNeeded(partitions); - if (!subscriptions.hasAllFetchPositions()) { - // if we still don't have offsets for all partitions, then we should either seek - // to the last committed position or reset using the auto reset policy + if (!subscriptions.hasAllFetchPositions(partitions)) { + // if we still don't have offsets for the given partitions, then we should either + // seek to the last committed position or reset using the auto reset policy // first refresh commits for all assigned partitions coordinator.refreshCommittedOffsetsIfNeeded(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 3a916a47d9c65..cd7f3077c49ca 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -208,7 +208,7 @@ public void resetOffsetsIfNeeded(Set partitions) { public void updateFetchPositions(Set partitions) { // reset the fetch position to the committed position for (TopicPartition tp : partitions) { - if (!subscriptions.isAssigned(tp) || subscriptions.isFetchable(tp)) + if (!subscriptions.isAssigned(tp) || subscriptions.hasValidPosition(tp)) continue; if (subscriptions.isOffsetResetNeeded(tp)) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index ad4f20b8b47ab..9e496ff08ce5b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -357,13 +357,17 @@ public OffsetResetStrategy resetStrategy(TopicPartition partition) { return assignedState(partition).resetStrategy; } - public boolean hasAllFetchPositions() { - for (TopicPartitionState state : assignment.partitionStateValues()) - if (!state.hasValidPosition()) + public boolean hasAllFetchPositions(Collection partitions) { + for (TopicPartition partition : partitions) + if (!hasValidPosition(partition)) return false; return true; } + public boolean hasAllFetchPositions() { + return hasAllFetchPositions(this.assignedPartitions()); + } + public Set missingFetchPositions() { Set missing = new HashSet<>(); for (PartitionStates.PartitionState state : assignment.partitionStates()) { @@ -385,6 +389,10 @@ public boolean isFetchable(TopicPartition tp) { return isAssigned(tp) && assignedState(tp).isFetchable(); } + public boolean hasValidPosition(TopicPartition tp) { + return isAssigned(tp) && assignedState(tp).hasValidPosition(); + } + public void pause(TopicPartition tp) { assignedState(tp).pause(); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index d4913df75a9c1..3e8310d410268 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -61,6 +61,7 @@ import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.test.MockConsumerInterceptor; import org.apache.kafka.test.MockMetricsReporter; import org.apache.kafka.test.TestUtils; @@ -1094,6 +1095,60 @@ public void testManualAssignmentChangeWithAutoCommitDisabled() { consumer.close(); } + @Test + public void testOffsetOfPausedPartitions() { + int rebalanceTimeoutMs = 60000; + int sessionTimeoutMs = 30000; + int heartbeatIntervalMs = 3000; + int autoCommitIntervalMs = 1000; + + Time time = new MockTime(); + Cluster cluster = TestUtils.singletonCluster(topic, 2); + Node node = cluster.nodes().get(0); + + Metadata metadata = new Metadata(0, Long.MAX_VALUE); + metadata.update(cluster, time.milliseconds()); + + MockClient client = new MockClient(time, metadata); + client.setNode(node); + PartitionAssignor assignor = new RangeAssignor(); + + final KafkaConsumer consumer = newConsumer(time, client, metadata, assignor, + rebalanceTimeoutMs, sessionTimeoutMs, heartbeatIntervalMs, true, autoCommitIntervalMs); + + // lookup coordinator + client.prepareResponseFrom(new GroupCoordinatorResponse(Errors.NONE.code(), node), node); + Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); + + // manual assignment + Set partitions = Utils.mkSet(tp0, tp1); + consumer.assign(partitions); + // verify consumer's assignment + assertTrue(consumer.assignment().equals(partitions)); + + consumer.pause(partitions); + consumer.seekToEnd(partitions); + + // fetch and verify committed offset of two partitions + Map offsets = new HashMap<>(); + offsets.put(tp0, 0L); + offsets.put(tp1, 0L); + + client.prepareResponseFrom(offsetResponse(offsets, Errors.NONE), coordinator); + assertEquals(0, consumer.committed(tp0).offset()); + assertEquals(0, consumer.committed(tp1).offset()); + + // fetch and verify consumer's position in the two partitions + client.prepareResponse(listOffsetsResponse(Collections.singletonMap(tp0, 3L), Errors.NONE.code())); + client.prepareResponse(listOffsetsResponse(Collections.singletonMap(tp1, 3L), Errors.NONE.code())); + assertEquals(3L, consumer.position(tp0)); + assertEquals(3L, consumer.position(tp1)); + + client.requests().clear(); + consumer.unsubscribe(); + consumer.close(); + } + @Test(expected = IllegalStateException.class) public void testPollWithNoSubscription() { KafkaConsumer consumer = newConsumer(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index cffa59f7a3989..bdd56c3e0f713 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -544,6 +544,68 @@ public void testUpdateFetchPositionDisconnect() { assertEquals(5, subscriptions.position(tp).longValue()); } + @Test + public void testUpdateFetchPositionOfPausedPartitionsRequiringOffsetReset() { + subscriptions.assignFromUser(singleton(tp)); + subscriptions.committed(tp, new OffsetAndMetadata(0)); + subscriptions.pause(tp); // paused partition does not have a valid position + subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); + + client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP), + listOffsetResponse(Errors.NONE, 1L, 10L)); + fetcher.updateFetchPositions(singleton(tp)); + + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertFalse(subscriptions.isFetchable(tp)); // because tp is paused + assertTrue(subscriptions.hasValidPosition(tp)); + assertEquals(10, subscriptions.position(tp).longValue()); + } + + @Test + public void testUpdateFetchPositionOfPausedPartitionsWithoutACommittedOffset() { + subscriptions.assignFromUser(singleton(tp)); + subscriptions.pause(tp); // paused partition does not have a valid position + + client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.EARLIEST_TIMESTAMP), + listOffsetResponse(Errors.NONE, 1L, 0L)); + fetcher.updateFetchPositions(singleton(tp)); + + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertFalse(subscriptions.isFetchable(tp)); // because tp is paused + assertTrue(subscriptions.hasValidPosition(tp)); + assertEquals(0, subscriptions.position(tp).longValue()); + } + + @Test + public void testUpdateFetchPositionOfPausedPartitionsWithoutAValidPosition() { + subscriptions.assignFromUser(singleton(tp)); + subscriptions.committed(tp, new OffsetAndMetadata(0)); + subscriptions.pause(tp); // paused partition does not have a valid position + subscriptions.seek(tp, 10); + + fetcher.updateFetchPositions(singleton(tp)); + + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertFalse(subscriptions.isFetchable(tp)); // because tp is paused + assertTrue(subscriptions.hasValidPosition(tp)); + assertEquals(10, subscriptions.position(tp).longValue()); + } + + @Test + public void testUpdateFetchPositionOfPausedPartitionsWithAValidPosition() { + subscriptions.assignFromUser(singleton(tp)); + subscriptions.committed(tp, new OffsetAndMetadata(0)); + subscriptions.seek(tp, 10); + subscriptions.pause(tp); // paused partition already has a valid position + + fetcher.updateFetchPositions(singleton(tp)); + + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertFalse(subscriptions.isFetchable(tp)); // because tp is paused + assertTrue(subscriptions.hasValidPosition(tp)); + assertEquals(10, subscriptions.position(tp).longValue()); + } + @Test public void testGetAllTopics() { // sending response before request, as getTopicMetadata is a blocking call From 3e1abac4c7d298d47567d23d6390eae4cd9ac1f8 Mon Sep 17 00:00:00 2001 From: Sandesh K Date: Thu, 19 Jan 2017 17:46:59 -0800 Subject: [PATCH 019/177] KAFKA-4432; Added support to supply custom message payloads to perf-producer script. Current implementation of ProducerPerformance creates static payload. This is not very useful in testing compression or when you want to test with production/custom payloads. So, we decided to add support for providing payload file as an input to producer perf test script. We made the following changes: 1. Added support to provide a payload file which can have the list of payloads that you actually want to send. 2. Moved payload generation inside the send loop for cases when payload file is provided. Following are the changes to how the producer-performance is evoked: 1. You must provide "--record-size" or "--payload-file" but not both. This is because, record size cannot be guaranteed when you are using custom events. e.g. ./kafka-producer-perf-test.sh --topic test_topic --num-records 100000 --producer-props bootstrap.servers=127.0.0.1:9092 acks=0 buffer.memory=33554432 compression.type=gzip batch.size=10240 linger.ms=10 --throughput -1 --payload-file ./test_payloads --payload-delimiter , 2. Earlier "--record-size" was a required config, now you must provide exactly one of "--record-size" or "--payload-file". Providing both will result in an error. 3. Support for an additional parameter "--payload-delimiter" has been added which defaults to "\n" Author: Sandesh K Reviewers: dan norwood , Jun Rao Closes #2158 from SandeshKarkera/PerfProducerChanges (cherry picked from commit a37bf5fffa4a607ded14ec055276d76f19df2d50) Signed-off-by: Jun Rao --- .../kafka/tools/ProducerPerformance.java | 82 ++++++++++++++++--- 1 file changed, 72 insertions(+), 10 deletions(-) diff --git a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java index a13d3ec68fc68..c277b83330767 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java +++ b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java @@ -3,9 +3,9 @@ * 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. @@ -14,11 +14,18 @@ import static net.sourceforge.argparse4j.impl.Arguments.store; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; + +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Properties; import java.util.Random; +import net.sourceforge.argparse4j.inf.MutuallyExclusiveGroup; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; @@ -42,15 +49,36 @@ public static void main(String[] args) throws Exception { /* parse args */ String topicName = res.getString("topic"); long numRecords = res.getLong("numRecords"); - int recordSize = res.getInt("recordSize"); + Integer recordSize = res.getInt("recordSize"); int throughput = res.getInt("throughput"); List producerProps = res.getList("producerConfig"); String producerConfig = res.getString("producerConfigFile"); + String payloadFilePath = res.getString("payloadFile"); + + // since default value gets printed with the help text, we are escaping \n there and replacing it with correct value here. + String payloadDelimiter = res.getString("payloadDelimiter").equals("\\n") ? "\n" : res.getString("payloadDelimiter"); if (producerProps == null && producerConfig == null) { throw new ArgumentParserException("Either --producer-props or --producer.config must be specified.", parser); } + List payloadByteList = new ArrayList<>(); + if (payloadFilePath != null) { + Path path = Paths.get(payloadFilePath); + System.out.println("Reading payloads from: " + path.toAbsolutePath()); + if (Files.notExists(path) || Files.size(path) == 0) { + throw new IllegalArgumentException("File does not exist or empty file provided."); + } + + String[] payloadList = new String(Files.readAllBytes(path), "UTF-8").split(payloadDelimiter); + + System.out.println("Number of messages read: " + payloadList.length); + + for (String payload : payloadList) { + payloadByteList.add(payload.getBytes(StandardCharsets.UTF_8)); + } + } + Properties props = new Properties(); if (producerConfig != null) { props.putAll(Utils.loadProps(producerConfig)); @@ -68,16 +96,24 @@ public static void main(String[] args) throws Exception { KafkaProducer producer = new KafkaProducer(props); /* setup perf test */ - byte[] payload = new byte[recordSize]; + byte[] payload = null; Random random = new Random(0); - for (int i = 0; i < payload.length; ++i) - payload[i] = (byte) (random.nextInt(26) + 65); - ProducerRecord record = new ProducerRecord<>(topicName, payload); + if (recordSize != null) { + payload = new byte[recordSize]; + for (int i = 0; i < payload.length; ++i) + payload[i] = (byte) (random.nextInt(26) + 65); + } + ProducerRecord record; Stats stats = new Stats(numRecords, 5000); long startMs = System.currentTimeMillis(); ThroughputThrottler throttler = new ThroughputThrottler(throughput, startMs); for (int i = 0; i < numRecords; i++) { + if (payloadFilePath != null) { + payload = payloadByteList.get(random.nextInt(payloadByteList.size())); + } + record = new ProducerRecord<>(topicName, payload); + long sendStartMs = System.currentTimeMillis(); Callback cb = stats.nextCompletion(sendStartMs, payload.length, stats); producer.send(record, cb); @@ -109,6 +145,11 @@ private static ArgumentParser argParser() { .defaultHelp(true) .description("This tool is used to verify the producer performance."); + MutuallyExclusiveGroup payloadOptions = parser + .addMutuallyExclusiveGroup() + .required(true) + .description("either --record-size or --payload-file must be specified but not both."); + parser.addArgument("--topic") .action(store()) .required(true) @@ -124,13 +165,34 @@ private static ArgumentParser argParser() { .dest("numRecords") .help("number of messages to produce"); - parser.addArgument("--record-size") + payloadOptions.addArgument("--record-size") .action(store()) - .required(true) + .required(false) .type(Integer.class) .metavar("RECORD-SIZE") .dest("recordSize") - .help("message size in bytes"); + .help("message size in bytes. Note that you must provide exactly one of --record-size or --payload-file."); + + payloadOptions.addArgument("--payload-file") + .action(store()) + .required(false) + .type(String.class) + .metavar("PAYLOAD-FILE") + .dest("payloadFile") + .help("file to read the message payloads from. This works only for UTF-8 encoded text files. " + + "Payloads will be read from this file and a payload will be randomly selected when sending messages. " + + "Note that you must provide exactly one of --record-size or --payload-file."); + + parser.addArgument("--payload-delimiter") + .action(store()) + .required(false) + .type(String.class) + .metavar("PAYLOAD-DELIMITER") + .dest("payloadDelimiter") + .setDefault("\\n") + .help("provides delimiter to be used when --payload-file is provided. " + + "Defaults to new line. " + + "Note that this parameter will be ignored if --payload-file is not provided."); parser.addArgument("--throughput") .action(store()) From 05237367d7a12f395c9b2a1c2b354e1564f21aae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 20 Jan 2017 11:07:57 -0800 Subject: [PATCH 020/177] MINOR: reduce verbosity of cache flushes MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This log message tends to be extremely verbose when state stores are being restored Author: Xavier Léauté Reviewers: Guozhang Wang Closes #2412 from xvrl/reduce-verbosity (cherry picked from commit 50e035dc341bb595decbac0851354a10c90b0e06) Signed-off-by: Guozhang Wang --- .../apache/kafka/streams/state/internals/NamedCache.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java index 6dc08f2247e8b..229ccec056649 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/NamedCache.java @@ -106,8 +106,10 @@ synchronized void flush() { private void flush(final LRUNode evicted) { numFlushes++; - log.debug("Named cache {} stats on flush: #hits={}, #misses={}, #overwrites={}, #flushes={}", - name, hits(), misses(), overwrites(), flushes()); + if (log.isTraceEnabled()) { + log.trace("Named cache {} stats on flush: #hits={}, #misses={}, #overwrites={}, #flushes={}", + name, hits(), misses(), overwrites(), flushes()); + } if (listener == null) { throw new IllegalArgumentException("No listener for namespace " + name + " registered with cache"); From c3f923cbf9f686bf42adda44d34a975167a13159 Mon Sep 17 00:00:00 2001 From: Shikhar Bhushan Date: Fri, 20 Jan 2017 16:15:15 -0800 Subject: [PATCH 021/177] KAFKA-3209: KIP-66: more single message transforms Renames `HoistToStruct` SMT to `HoistField`. Adds the following SMTs: `ExtractField` `MaskField` `RegexRouter` `ReplaceField` `SetSchemaMetadata` `ValueToKey` Adds HTML doc generation and updates to `connect.html`. Author: Shikhar Bhushan Reviewers: Ewen Cheslack-Postava Closes #2374 from shikhar/more-smt --- build.gradle | 9 +- .../connect/tools/TransformationDoc.java | 87 +++++++ .../connect/transforms/ExtractField.java | 114 +++++++++ .../{HoistToStruct.java => HoistField.java} | 46 ++-- .../kafka/connect/transforms/InsertField.java | 170 ++++++------- .../kafka/connect/transforms/MaskField.java | 172 +++++++++++++ .../kafka/connect/transforms/RegexRouter.java | 75 ++++++ .../connect/transforms/ReplaceField.java | 230 ++++++++++++++++++ .../connect/transforms/SetSchemaMetadata.java | 124 ++++++++++ .../connect/transforms/TimestampRouter.java | 30 +-- .../kafka/connect/transforms/ValueToKey.java | 111 +++++++++ .../util/NonEmptyListValidator.java | 39 +++ .../transforms/util/RegexValidator.java | 41 ++++ .../connect/transforms/util/Requirements.java | 61 +++++ .../connect/transforms/util/SchemaUtil.java | 40 +++ .../connect/transforms/ExtractFieldTest.java | 59 +++++ ...tToStructTest.java => HoistFieldTest.java} | 19 +- .../connect/transforms/MaskFieldTest.java | 156 ++++++++++++ .../connect/transforms/RegexRouterTest.java | 70 ++++++ .../connect/transforms/ReplaceFieldTest.java | 92 +++++++ .../transforms/SetSchemaMetadataTest.java | 67 +++++ .../connect/transforms/ValueToKeyTest.java | 87 +++++++ docs/connect.html | 16 ++ .../tests/connect/connect_distributed_test.py | 2 +- 24 files changed, 1779 insertions(+), 138 deletions(-) create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java rename connect/transforms/src/main/java/org/apache/kafka/connect/transforms/{HoistToStruct.java => HoistField.java} (73%) create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/RegexRouter.java create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/RegexValidator.java create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java create mode 100644 connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ExtractFieldTest.java rename connect/transforms/src/test/java/org/apache/kafka/connect/transforms/{HoistToStructTest.java => HoistFieldTest.java} (71%) create mode 100644 connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java create mode 100644 connect/transforms/src/test/java/org/apache/kafka/connect/transforms/RegexRouterTest.java create mode 100644 connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java create mode 100644 connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java create mode 100644 connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java diff --git a/build.gradle b/build.gradle index 6f4d2501f7345..e191a51c3b89a 100644 --- a/build.gradle +++ b/build.gradle @@ -508,7 +508,7 @@ project(':core') { task siteDocsTar(dependsOn: ['genProtocolErrorDocs', 'genProtocolApiKeyDocs', 'genProtocolMessageDocs', 'genProducerConfigDocs', 'genConsumerConfigDocs', 'genKafkaConfigDocs', - 'genTopicConfigDocs', ':connect:runtime:genConnectConfigDocs', + 'genTopicConfigDocs', ':connect:runtime:genConnectConfigDocs', ':connect:runtime:genConnectTransformationDocs', ':streams:genStreamsConfigDocs'], type: Tar) { classifier = 'site-docs' compression = Compression.GZIP @@ -948,6 +948,13 @@ project(':connect:runtime') { if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } standardOutput = new File(generatedDocsDir, "connect_config.html").newOutputStream() } + + task genConnectTransformationDocs(type: JavaExec) { + classpath = sourceSets.main.runtimeClasspath + main = 'org.apache.kafka.connect.tools.TransformationDoc' + if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } + standardOutput = new File(generatedDocsDir, "connect_transforms.html").newOutputStream() + } } project(':connect:file') { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java new file mode 100644 index 0000000000000..6746042baa743 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java @@ -0,0 +1,87 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.tools; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.transforms.ExtractField; +import org.apache.kafka.connect.transforms.HoistField; +import org.apache.kafka.connect.transforms.InsertField; +import org.apache.kafka.connect.transforms.MaskField; +import org.apache.kafka.connect.transforms.RegexRouter; +import org.apache.kafka.connect.transforms.ReplaceField; +import org.apache.kafka.connect.transforms.SetSchemaMetadata; +import org.apache.kafka.connect.transforms.TimestampRouter; +import org.apache.kafka.connect.transforms.ValueToKey; + +import java.io.PrintStream; +import java.util.Arrays; +import java.util.List; + +public class TransformationDoc { + + private static final class DocInfo { + final String transformationName; + final String overview; + final ConfigDef configDef; + + private DocInfo(String transformationName, String overview, ConfigDef configDef) { + this.transformationName = transformationName; + this.overview = overview; + this.configDef = configDef; + } + } + + private static final List TRANSFORMATIONS = Arrays.asList( + new DocInfo(InsertField.class.getName(), InsertField.OVERVIEW_DOC, InsertField.CONFIG_DEF), + new DocInfo(ReplaceField.class.getName(), ReplaceField.OVERVIEW_DOC, ReplaceField.CONFIG_DEF), + new DocInfo(MaskField.class.getName(), MaskField.OVERVIEW_DOC, MaskField.CONFIG_DEF), + new DocInfo(ValueToKey.class.getName(), ValueToKey.OVERVIEW_DOC, ValueToKey.CONFIG_DEF), + new DocInfo(HoistField.class.getName(), HoistField.OVERVIEW_DOC, HoistField.CONFIG_DEF), + new DocInfo(ExtractField.class.getName(), ExtractField.OVERVIEW_DOC, ExtractField.CONFIG_DEF), + new DocInfo(SetSchemaMetadata.class.getName(), SetSchemaMetadata.OVERVIEW_DOC, SetSchemaMetadata.CONFIG_DEF), + new DocInfo(TimestampRouter.class.getName(), TimestampRouter.OVERVIEW_DOC, TimestampRouter.CONFIG_DEF), + new DocInfo(RegexRouter.class.getName(), RegexRouter.OVERVIEW_DOC, RegexRouter.CONFIG_DEF) + ); + + private static void printTransformationHtml(PrintStream out, DocInfo docInfo) { + out.println("

      "); + + out.print("
      "); + out.print(docInfo.transformationName); + out.println("
      "); + + out.println(docInfo.overview); + + out.println("

      "); + + out.println(docInfo.configDef.toHtmlTable()); + + out.println("

      "); + } + + private static void printHtml(PrintStream out) throws NoSuchFieldException, IllegalAccessException, InstantiationException { + for (final DocInfo docInfo : TRANSFORMATIONS) { + printTransformationHtml(out, docInfo); + } + } + + public static void main(String... args) throws Exception { + printHtml(System.out); + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java new file mode 100644 index 0000000000000..b7063136e9df1 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java @@ -0,0 +1,114 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.transforms; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +import java.util.Map; + +import static org.apache.kafka.connect.transforms.util.Requirements.requireMap; +import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct; + +public abstract class ExtractField> implements Transformation { + + public static final String OVERVIEW_DOC = + "Extract the specified field from a Struct when schema present, or a Map in the case of schemaless data." + + "

      Use the concrete transformation type designed for the record key (" + Key.class.getCanonicalName() + ") " + + "or value (" + Value.class.getCanonicalName() + ")."; + + private static final String FIELD_CONFIG = "field"; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(FIELD_CONFIG, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.Importance.MEDIUM, "Field name to extract."); + + private static final String PURPOSE = "field extraction"; + + private String fieldName; + + @Override + public void configure(Map props) { + final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); + fieldName = config.getString(FIELD_CONFIG); + } + + @Override + public R apply(R record) { + final Schema schema = operatingSchema(record); + if (schema == null) { + final Map value = requireMap(operatingValue(record), PURPOSE); + return newRecord(record, null, value.get(fieldName)); + } else { + final Struct value = requireStruct(operatingValue(record), PURPOSE); + return newRecord(record, schema.field(fieldName).schema(), value.get(fieldName)); + } + } + + @Override + public void close() { + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + protected abstract Schema operatingSchema(R record); + + protected abstract Object operatingValue(R record); + + protected abstract R newRecord(R record, Schema updatedSchema, Object updatedValue); + + public static class Key> extends ExtractField { + @Override + protected Schema operatingSchema(R record) { + return record.keySchema(); + } + + @Override + protected Object operatingValue(R record) { + return record.key(); + } + + @Override + protected R newRecord(R record, Schema updatedSchema, Object updatedValue) { + return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, updatedValue, record.valueSchema(), record.value(), record.timestamp()); + } + } + + public static class Value> extends ExtractField { + @Override + protected Schema operatingSchema(R record) { + return record.valueSchema(); + } + + @Override + protected Object operatingValue(R record) { + return record.value(); + } + + @Override + protected R newRecord(R record, Schema updatedSchema, Object updatedValue) { + return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), updatedSchema, updatedValue, record.timestamp()); + } + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistField.java similarity index 73% rename from connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java rename to connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistField.java index c2726ca84378b..1f2ed7c8e626f 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistField.java @@ -27,15 +27,21 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.transforms.util.SimpleConfig; +import java.util.Collections; import java.util.Map; -public abstract class HoistToStruct> implements Transformation { +public abstract class HoistField> implements Transformation { - public static final String FIELD_CONFIG = "field"; + public static final String OVERVIEW_DOC = + "Wrap data using the specified field name in a Struct when schema present, or a Map in the case of schemaless data." + + "

      Use the concrete transformation type designed for the record key (" + Key.class.getCanonicalName() + ") " + + "or value (" + Value.class.getCanonicalName() + ")."; - private static final ConfigDef CONFIG_DEF = new ConfigDef() + private static final String FIELD_CONFIG = "field"; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() .define(FIELD_CONFIG, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.Importance.MEDIUM, - "Field name for the single field that will be created in the resulting Struct."); + "Field name for the single field that will be created in the resulting Struct or Map."); private Cache schemaUpdateCache; @@ -53,15 +59,19 @@ public R apply(R record) { final Schema schema = operatingSchema(record); final Object value = operatingValue(record); - Schema updatedSchema = schemaUpdateCache.get(schema); - if (updatedSchema == null) { - updatedSchema = SchemaBuilder.struct().field(fieldName, schema).build(); - schemaUpdateCache.put(schema, updatedSchema); - } + if (schema == null) { + return newRecord(record, null, Collections.singletonMap(fieldName, value)); + } else { + Schema updatedSchema = schemaUpdateCache.get(schema); + if (updatedSchema == null) { + updatedSchema = SchemaBuilder.struct().field(fieldName, schema).build(); + schemaUpdateCache.put(schema, updatedSchema); + } - final Struct updatedValue = new Struct(updatedSchema).put(fieldName, value); + final Struct updatedValue = new Struct(updatedSchema).put(fieldName, value); - return newRecord(record, updatedSchema, updatedValue); + return newRecord(record, updatedSchema, updatedValue); + } } @Override @@ -80,11 +90,7 @@ public ConfigDef config() { protected abstract R newRecord(R record, Schema updatedSchema, Object updatedValue); - /** - * Wraps the record key in a {@link org.apache.kafka.connect.data.Struct} with specified field name. - */ - public static class Key> extends HoistToStruct { - + public static class Key> extends HoistField { @Override protected Schema operatingSchema(R record) { return record.keySchema(); @@ -99,14 +105,9 @@ protected Object operatingValue(R record) { protected R newRecord(R record, Schema updatedSchema, Object updatedValue) { return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, updatedValue, record.valueSchema(), record.value(), record.timestamp()); } - } - /** - * Wraps the record value in a {@link org.apache.kafka.connect.data.Struct} with specified field name. - */ - public static class Value> extends HoistToStruct { - + public static class Value> extends HoistField { @Override protected Schema operatingSchema(R record) { return record.valueSchema(); @@ -121,7 +122,6 @@ protected Object operatingValue(R record) { protected R newRecord(R record, Schema updatedSchema, Object updatedValue) { return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), updatedSchema, updatedValue, record.timestamp()); } - } } diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java index d67fea0893237..f32d6ed2e9465 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java @@ -21,23 +21,32 @@ import org.apache.kafka.common.cache.LRUCache; import org.apache.kafka.common.cache.SynchronizedCache; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.connect.connector.ConnectRecord; import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.data.Timestamp; -import org.apache.kafka.connect.errors.DataException; -import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.transforms.util.SimpleConfig; +import org.apache.kafka.connect.transforms.util.SchemaUtil; import java.util.Date; import java.util.HashMap; import java.util.Map; +import static org.apache.kafka.connect.transforms.util.Requirements.requireMap; +import static org.apache.kafka.connect.transforms.util.Requirements.requireSinkRecord; +import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct; + public abstract class InsertField> implements Transformation { - public interface Keys { + public static final String OVERVIEW_DOC = + "Insert field(s) using attributes from the record metadata or a configured static value." + + "

      Use the concrete transformation type designed for the record key (" + Key.class.getCanonicalName() + ") " + + "or value (" + Value.class.getCanonicalName() + ")."; + + private interface ConfigName { String TOPIC_FIELD = "topic.field"; String PARTITION_FIELD = "partition.field"; String OFFSET_FIELD = "offset.field"; @@ -46,22 +55,24 @@ public interface Keys { String STATIC_VALUE = "static.value"; } - private static final String OPTIONALITY_DOC = "Suffix with '!' to make this a required field, or '?' to keep it optional (the default)."; - - private static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(Keys.TOPIC_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, - "Field name for Kafka topic.\n" + OPTIONALITY_DOC) - .define(Keys.PARTITION_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, - "Field name for Kafka partition.\n" + OPTIONALITY_DOC) - .define(Keys.OFFSET_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, - "Field name for Kafka offset - only applicable to sink connectors.\n" + OPTIONALITY_DOC) - .define(Keys.TIMESTAMP_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, - "Field name for record timestamp.\n" + OPTIONALITY_DOC) - .define(Keys.STATIC_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, - "Field name for static data field.\n" + OPTIONALITY_DOC) - .define(Keys.STATIC_VALUE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, + private static final String OPTIONALITY_DOC = "Suffix with ! to make this a required field, or ? to keep it optional (the default)."; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(ConfigName.TOPIC_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, + "Field name for Kafka topic. " + OPTIONALITY_DOC) + .define(ConfigName.PARTITION_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, + "Field name for Kafka partition. " + OPTIONALITY_DOC) + .define(ConfigName.OFFSET_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, + "Field name for Kafka offset - only applicable to sink connectors.
      " + OPTIONALITY_DOC) + .define(ConfigName.TIMESTAMP_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, + "Field name for record timestamp. " + OPTIONALITY_DOC) + .define(ConfigName.STATIC_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, + "Field name for static data field. " + OPTIONALITY_DOC) + .define(ConfigName.STATIC_VALUE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, "Static field value, if field name configured."); + private static final String PURPOSE = "field insertion"; + private static final Schema OPTIONAL_TIMESTAMP_SCHEMA = Timestamp.builder().optional().build(); private static final class InsertionSpec { @@ -91,46 +102,42 @@ public static InsertionSpec parse(String spec) { private InsertionSpec timestampField; private InsertionSpec staticField; private String staticValue; - private boolean applicable; private Cache schemaUpdateCache; @Override public void configure(Map props) { final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); - topicField = InsertionSpec.parse(config.getString(Keys.TOPIC_FIELD)); - partitionField = InsertionSpec.parse(config.getString(Keys.PARTITION_FIELD)); - offsetField = InsertionSpec.parse(config.getString(Keys.OFFSET_FIELD)); - timestampField = InsertionSpec.parse(config.getString(Keys.TIMESTAMP_FIELD)); - staticField = InsertionSpec.parse(config.getString(Keys.STATIC_FIELD)); - staticValue = config.getString(Keys.STATIC_VALUE); - applicable = topicField != null || partitionField != null || offsetField != null || timestampField != null; + topicField = InsertionSpec.parse(config.getString(ConfigName.TOPIC_FIELD)); + partitionField = InsertionSpec.parse(config.getString(ConfigName.PARTITION_FIELD)); + offsetField = InsertionSpec.parse(config.getString(ConfigName.OFFSET_FIELD)); + timestampField = InsertionSpec.parse(config.getString(ConfigName.TIMESTAMP_FIELD)); + staticField = InsertionSpec.parse(config.getString(ConfigName.STATIC_FIELD)); + staticValue = config.getString(ConfigName.STATIC_VALUE); + + if (topicField == null && partitionField == null && offsetField == null && timestampField == null && staticField == null) { + throw new ConfigException("No field insertion configured"); + } + + if (staticField != null && staticValue == null) { + throw new ConfigException(ConfigName.STATIC_VALUE, null, "No value specified for static field: " + staticField); + } schemaUpdateCache = new SynchronizedCache<>(new LRUCache(16)); } @Override public R apply(R record) { - if (!applicable) return record; - - final Schema schema = operatingSchema(record); - final Object value = operatingValue(record); - - if (value == null) - throw new DataException("null value"); - - if (schema == null) { - if (!(value instanceof Map)) - throw new DataException("Can only operate on Map value in schemaless mode: " + value.getClass().getName()); - return applySchemaless(record, (Map) value); + if (operatingSchema(record) == null) { + return applySchemaless(record); } else { - if (schema.type() != Schema.Type.STRUCT) - throw new DataException("Can only operate on Struct types: " + value.getClass().getName()); - return applyWithSchema(record, schema, (Struct) value); + return applyWithSchema(record); } } - private R applySchemaless(R record, Map value) { + private R applySchemaless(R record) { + final Map value = requireMap(operatingValue(record), PURPOSE); + final Map updatedValue = new HashMap<>(value); if (topicField != null) { @@ -140,9 +147,7 @@ private R applySchemaless(R record, Map value) { updatedValue.put(partitionField.name, record.kafkaPartition()); } if (offsetField != null) { - if (!(record instanceof SinkRecord)) - throw new DataException("Offset insertion is only supported for sink connectors, record is of type: " + record.getClass()); - updatedValue.put(offsetField.name, ((SinkRecord) record).kafkaOffset()); + updatedValue.put(offsetField.name, requireSinkRecord(record, PURPOSE).kafkaOffset()); } if (timestampField != null && record.timestamp() != null) { updatedValue.put(timestampField.name, record.timestamp()); @@ -150,36 +155,46 @@ private R applySchemaless(R record, Map value) { if (staticField != null && staticValue != null) { updatedValue.put(staticField.name, staticValue); } + return newRecord(record, null, updatedValue); } - private R applyWithSchema(R record, Schema schema, Struct value) { - Schema updatedSchema = schemaUpdateCache.get(schema); + private R applyWithSchema(R record) { + final Struct value = requireStruct(operatingValue(record), PURPOSE); + + Schema updatedSchema = schemaUpdateCache.get(value.schema()); if (updatedSchema == null) { - updatedSchema = makeUpdatedSchema(schema); - schemaUpdateCache.put(schema, updatedSchema); + updatedSchema = makeUpdatedSchema(value.schema()); + schemaUpdateCache.put(value.schema(), updatedSchema); } final Struct updatedValue = new Struct(updatedSchema); - copyFields(value, updatedValue); + for (Field field : value.schema().fields()) { + updatedValue.put(field.name(), value.get(field)); + } - insertFields(record, updatedValue); + if (topicField != null) { + updatedValue.put(topicField.name, record.topic()); + } + if (partitionField != null && record.kafkaPartition() != null) { + updatedValue.put(partitionField.name, record.kafkaPartition()); + } + if (offsetField != null) { + updatedValue.put(offsetField.name, requireSinkRecord(record, PURPOSE).kafkaOffset()); + } + if (timestampField != null && record.timestamp() != null) { + updatedValue.put(timestampField.name, new Date(record.timestamp())); + } + if (staticField != null && staticValue != null) { + updatedValue.put(staticField.name, staticValue); + } return newRecord(record, updatedSchema, updatedValue); } private Schema makeUpdatedSchema(Schema schema) { - final SchemaBuilder builder = SchemaBuilder.struct(); - - builder.name(schema.name()); - builder.version(schema.version()); - builder.doc(schema.doc()); - - final Map params = schema.parameters(); - if (params != null) { - builder.parameters(params); - } + final SchemaBuilder builder = SchemaUtil.copySchemaBasics(schema, SchemaBuilder.struct()); for (Field field : schema.fields()) { builder.field(field.name(), field.schema()); @@ -204,33 +219,6 @@ private Schema makeUpdatedSchema(Schema schema) { return builder.build(); } - private void copyFields(Struct value, Struct updatedValue) { - for (Field field : value.schema().fields()) { - updatedValue.put(field.name(), value.get(field)); - } - } - - private void insertFields(R record, Struct value) { - if (topicField != null) { - value.put(topicField.name, record.topic()); - } - if (partitionField != null && record.kafkaPartition() != null) { - value.put(partitionField.name, record.kafkaPartition()); - } - if (offsetField != null) { - if (!(record instanceof SinkRecord)) { - throw new DataException("Offset insertion is only supported for sink connectors, record is of type: " + record.getClass()); - } - value.put(offsetField.name, ((SinkRecord) record).kafkaOffset()); - } - if (timestampField != null && record.timestamp() != null) { - value.put(timestampField.name, new Date(record.timestamp())); - } - if (staticField != null && staticValue != null) { - value.put(staticField.name, staticValue); - } - } - @Override public void close() { schemaUpdateCache = null; @@ -247,10 +235,6 @@ public ConfigDef config() { protected abstract R newRecord(R record, Schema updatedSchema, Object updatedValue); - /** - * This transformation allows inserting configured attributes of the record metadata as fields in the record key. - * It also allows adding a static data field. - */ public static class Key> extends InsertField { @Override @@ -270,10 +254,6 @@ protected R newRecord(R record, Schema updatedSchema, Object updatedValue) { } - /** - * This transformation allows inserting configured attributes of the record metadata as fields in the record value. - * It also allows adding a static data field. - */ public static class Value> extends InsertField { @Override diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java new file mode 100644 index 0000000000000..d7ef2aa8429ad --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java @@ -0,0 +1,172 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.transforms; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.kafka.connect.transforms.util.Requirements.requireMap; +import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct; + +public abstract class MaskField> implements Transformation { + + public static final String OVERVIEW_DOC = + "Mask specified fields with a valid null value for the field type (i.e. 0, false, empty string, and so on)." + + "

      Use the concrete transformation type designed for the record key (" + Key.class.getCanonicalName() + ") " + + "or value (" + Value.class.getCanonicalName() + ")."; + + private static final String FIELDS_CONFIG = "fields"; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, new NonEmptyListValidator(), ConfigDef.Importance.HIGH, "Names of fields to mask."); + + private static final String PURPOSE = "mask fields"; + + private static final Map, Object> PRIMITIVE_VALUE_MAPPING = new HashMap<>(); + + static { + PRIMITIVE_VALUE_MAPPING.put(Boolean.class, Boolean.FALSE); + PRIMITIVE_VALUE_MAPPING.put(Byte.class, (byte) 0); + PRIMITIVE_VALUE_MAPPING.put(Short.class, (short) 0); + PRIMITIVE_VALUE_MAPPING.put(Integer.class, 0); + PRIMITIVE_VALUE_MAPPING.put(Long.class, 0L); + PRIMITIVE_VALUE_MAPPING.put(Float.class, 0f); + PRIMITIVE_VALUE_MAPPING.put(Double.class, 0d); + PRIMITIVE_VALUE_MAPPING.put(BigInteger.class, BigInteger.ZERO); + PRIMITIVE_VALUE_MAPPING.put(BigDecimal.class, BigDecimal.ZERO); + PRIMITIVE_VALUE_MAPPING.put(Date.class, new Date(0)); + PRIMITIVE_VALUE_MAPPING.put(String.class, ""); + } + + private Set maskedFields; + + @Override + public void configure(Map props) { + final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); + maskedFields = new HashSet<>(config.getList(FIELDS_CONFIG)); + } + + @Override + public R apply(R record) { + if (operatingSchema(record) == null) { + return applySchemaless(record); + } else { + return applyWithSchema(record); + } + } + + private R applySchemaless(R record) { + final Map value = requireMap(operatingValue(record), PURPOSE); + final HashMap updatedValue = new HashMap<>(value); + for (String field : maskedFields) { + updatedValue.put(field, masked(value.get(field))); + } + return newRecord(record, updatedValue); + } + + private R applyWithSchema(R record) { + final Struct value = requireStruct(operatingValue(record), PURPOSE); + final Struct updatedValue = new Struct(value.schema()); + for (Field field : value.schema().fields()) { + final Object origFieldValue = value.get(field); + updatedValue.put(field, maskedFields.contains(field.name()) ? masked(origFieldValue) : origFieldValue); + } + return newRecord(record, updatedValue); + } + + private static Object masked(Object value) { + if (value == null) + return null; + Object maskedValue = PRIMITIVE_VALUE_MAPPING.get(value.getClass()); + if (maskedValue == null) { + if (value instanceof List) + maskedValue = Collections.emptyList(); + else if (value instanceof Map) + maskedValue = Collections.emptyMap(); + else + throw new DataException("Cannot mask value of type: " + value.getClass()); + } + return maskedValue; + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() { + } + + protected abstract Schema operatingSchema(R record); + + protected abstract Object operatingValue(R record); + + protected abstract R newRecord(R base, Object value); + + public static final class Key> extends MaskField { + @Override + protected Schema operatingSchema(R record) { + return record.keySchema(); + } + + @Override + protected Object operatingValue(R record) { + return record.key(); + } + + @Override + protected R newRecord(R record, Object updatedValue) { + return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), updatedValue, record.valueSchema(), record.value(), record.timestamp()); + } + } + + public static final class Value> extends MaskField { + @Override + protected Schema operatingSchema(R record) { + return record.valueSchema(); + } + + @Override + protected Object operatingValue(R record) { + return record.value(); + } + + @Override + protected R newRecord(R record, Object updatedValue) { + return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), record.valueSchema(), updatedValue, record.timestamp()); + } + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/RegexRouter.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/RegexRouter.java new file mode 100644 index 0000000000000..f16560e3c5748 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/RegexRouter.java @@ -0,0 +1,75 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.transforms; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.transforms.util.RegexValidator; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class RegexRouter> implements Transformation { + + public static final String OVERVIEW_DOC = "Update the record topic using the configured regular expression and replacement string." + + "

      Under the hood, the regex is compiled to a java.util.regex.Pattern. " + + "If the pattern matches the input topic, java.util.regex.Matcher#replaceFirst() is used with the replacement string to obtain the new topic."; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(ConfigName.REGEX, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, new RegexValidator(), ConfigDef.Importance.HIGH, + "Regular expression to use for matching.") + .define(ConfigName.REPLACEMENT, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.Importance.HIGH, + "Replacement string."); + + private interface ConfigName { + String REGEX = "regex"; + String REPLACEMENT = "replacement"; + } + + private Pattern regex; + private String replacement; + + @Override + public void configure(Map props) { + final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); + regex = Pattern.compile(config.getString(ConfigName.REGEX)); + replacement = config.getString(ConfigName.REPLACEMENT); + } + + @Override + public R apply(R record) { + final Matcher matcher = regex.matcher(record.topic()); + if (matcher.matches()) { + final String topic = matcher.replaceFirst(replacement); + return record.newRecord(topic, record.kafkaPartition(), record.keySchema(), record.key(), record.valueSchema(), record.value(), record.timestamp()); + } + return record; + } + + @Override + public void close() { + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java new file mode 100644 index 0000000000000..6faf84250af07 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java @@ -0,0 +1,230 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.transforms; + +import org.apache.kafka.common.cache.Cache; +import org.apache.kafka.common.cache.LRUCache; +import org.apache.kafka.common.cache.SynchronizedCache; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.transforms.util.SchemaUtil; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.kafka.connect.transforms.util.Requirements.requireMap; +import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct; + +public abstract class ReplaceField> implements Transformation { + + public static final String OVERVIEW_DOC = "Filter or rename fields."; + + interface ConfigName { + String BLACKLIST = "blacklist"; + String WHITELIST = "whitelist"; + String RENAME = "renames"; + } + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(ConfigName.BLACKLIST, ConfigDef.Type.LIST, Collections.emptyList(), ConfigDef.Importance.MEDIUM, + "Fields to exclude. This takes precedence over the whitelist.") + .define(ConfigName.WHITELIST, ConfigDef.Type.LIST, Collections.emptyList(), ConfigDef.Importance.MEDIUM, + "Fields to include. If specified, only these fields will be used.") + .define(ConfigName.RENAME, ConfigDef.Type.LIST, Collections.emptyList(), new ConfigDef.Validator() { + @Override + public void ensureValid(String name, Object value) { + parseRenameMappings((List) value); + } + + @Override + public String toString() { + return "list of colon-delimited pairs, e.g. foo:bar,abc:xyz"; + } + }, ConfigDef.Importance.MEDIUM, "Field rename mappings."); + + private static final String PURPOSE = "field replacement"; + + private List blacklist; + private List whitelist; + private Map renames; + private Map reverseRenames; + + private Cache schemaUpdateCache; + + @Override + public void configure(Map configs) { + final SimpleConfig config = new SimpleConfig(CONFIG_DEF, configs); + blacklist = config.getList(ConfigName.BLACKLIST); + whitelist = config.getList(ConfigName.WHITELIST); + renames = parseRenameMappings(config.getList(ConfigName.RENAME)); + reverseRenames = invert(renames); + + schemaUpdateCache = new SynchronizedCache<>(new LRUCache(16)); + } + + static Map parseRenameMappings(List mappings) { + final Map m = new HashMap<>(); + for (String mapping : mappings) { + final String[] parts = mapping.split(":"); + if (parts.length != 2) { + throw new ConfigException(ConfigName.RENAME, mappings, "Invalid rename mapping: " + mapping); + } + m.put(parts[0], parts[1]); + } + return m; + } + + static Map invert(Map source) { + final Map m = new HashMap<>(); + for (Map.Entry e : source.entrySet()) { + m.put(e.getValue(), e.getKey()); + } + return m; + } + + boolean filter(String fieldName) { + return !blacklist.contains(fieldName) && (whitelist.isEmpty() || whitelist.contains(fieldName)); + } + + String renamed(String fieldName) { + final String mapping = renames.get(fieldName); + return mapping == null ? fieldName : mapping; + } + + String reverseRenamed(String fieldName) { + final String mapping = reverseRenames.get(fieldName); + return mapping == null ? fieldName : mapping; + } + + @Override + public R apply(R record) { + if (operatingSchema(record) == null) { + return applySchemaless(record); + } else { + return applyWithSchema(record); + } + } + + private R applySchemaless(R record) { + final Map value = requireMap(operatingValue(record), PURPOSE); + + final Map updatedValue = new HashMap<>(value.size()); + + for (Map.Entry e : value.entrySet()) { + final String fieldName = e.getKey(); + if (filter(fieldName)) { + final Object fieldValue = e.getValue(); + updatedValue.put(renamed(fieldName), fieldValue); + } + } + + return newRecord(record, null, updatedValue); + } + + private R applyWithSchema(R record) { + final Struct value = requireStruct(operatingValue(record), PURPOSE); + + Schema updatedSchema = schemaUpdateCache.get(value.schema()); + if (updatedSchema == null) { + updatedSchema = makeUpdatedSchema(value.schema()); + schemaUpdateCache.put(value.schema(), updatedSchema); + } + + final Struct updatedValue = new Struct(updatedSchema); + + for (Field field : updatedSchema.fields()) { + final Object fieldValue = value.get(reverseRenamed(field.name())); + updatedValue.put(field.name(), fieldValue); + } + + return newRecord(record, updatedSchema, updatedValue); + } + + private Schema makeUpdatedSchema(Schema schema) { + final SchemaBuilder builder = SchemaUtil.copySchemaBasics(schema, SchemaBuilder.struct()); + for (Field field : schema.fields()) { + if (filter(field.name())) { + builder.field(renamed(field.name()), field.schema()); + } + } + return builder.build(); + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() { + schemaUpdateCache = null; + } + + protected abstract Schema operatingSchema(R record); + + protected abstract Object operatingValue(R record); + + protected abstract R newRecord(R record, Schema updatedSchema, Object updatedValue); + + public static class Key> extends ReplaceField { + + @Override + protected Schema operatingSchema(R record) { + return record.keySchema(); + } + + @Override + protected Object operatingValue(R record) { + return record.key(); + } + + @Override + protected R newRecord(R record, Schema updatedSchema, Object updatedValue) { + return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, updatedValue, record.valueSchema(), record.value(), record.timestamp()); + } + + } + + public static class Value> extends ReplaceField { + + @Override + protected Schema operatingSchema(R record) { + return record.valueSchema(); + } + + @Override + protected Object operatingValue(R record) { + return record.value(); + } + + @Override + protected R newRecord(R record, Schema updatedSchema, Object updatedValue) { + return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), updatedSchema, updatedValue, record.timestamp()); + } + + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java new file mode 100644 index 0000000000000..f3076b445719b --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java @@ -0,0 +1,124 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.transforms; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.ConnectSchema; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +import java.util.Map; + +import static org.apache.kafka.connect.transforms.util.Requirements.requireSchema; + +public abstract class SetSchemaMetadata> implements Transformation { + + public static final String OVERVIEW_DOC = + "Set the schema name, version or both on the record's key (" + Key.class.getCanonicalName() + ")" + + " or value (" + Value.class.getCanonicalName() + ") schema."; + + private interface ConfigName { + String SCHEMA_NAME = "schema.name"; + String SCHEMA_VERSION = "schema.version"; + } + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(ConfigName.SCHEMA_NAME, ConfigDef.Type.STRING, null, ConfigDef.Importance.HIGH, "Schema name to set.") + .define(ConfigName.SCHEMA_VERSION, ConfigDef.Type.INT, null, ConfigDef.Importance.HIGH, "Schema version to set."); + + private String schemaName; + private Integer schemaVersion; + + @Override + public void configure(Map configs) { + final SimpleConfig config = new SimpleConfig(CONFIG_DEF, configs); + schemaName = config.getString(ConfigName.SCHEMA_NAME); + schemaVersion = config.getInt(ConfigName.SCHEMA_VERSION); + + if (schemaName == null && schemaVersion == null) { + throw new ConfigException("Neither schema name nor version configured"); + } + } + + @Override + public R apply(R record) { + final Schema schema = operatingSchema(record); + requireSchema(schema, "updating schema metadata"); + final boolean isArray = schema.type() == Schema.Type.ARRAY; + final boolean isMap = schema.type() == Schema.Type.MAP; + final Schema updatedSchema = new ConnectSchema( + schema.type(), + schema.isOptional(), + schema.defaultValue(), + schemaName != null ? schemaName : schema.name(), + schemaVersion != null ? schemaVersion : schema.version(), + schema.doc(), + schema.parameters(), + schema.fields(), + isMap ? schema.keySchema() : null, + isMap || isArray ? schema.valueSchema() : null + ); + return newRecord(record, updatedSchema); + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() { + } + + protected abstract Schema operatingSchema(R record); + + protected abstract R newRecord(R record, Schema updatedSchema); + + /** + * Set the schema name, version or both on the record's key schema. + */ + public static class Key> extends SetSchemaMetadata { + @Override + protected Schema operatingSchema(R record) { + return record.keySchema(); + } + + @Override + protected R newRecord(R record, Schema updatedSchema) { + return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, record.key(), record.valueSchema(), record.value(), record.timestamp()); + } + } + + /** + * Set the schema name, version or both on the record's value schema. + */ + public static class Value> extends SetSchemaMetadata { + @Override + protected Schema operatingSchema(R record) { + return record.valueSchema(); + } + + @Override + protected R newRecord(R record, Schema updatedSchema) { + return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), updatedSchema, record.value(), record.timestamp()); + } + } + +} \ No newline at end of file diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java index 1dd5345a6091a..f917a8d65f09a 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java @@ -27,25 +27,25 @@ import java.util.Map; import java.util.TimeZone; -/** - * This transformation facilitates updating the record's topic field as a function of the original topic value and the record timestamp. - *

      - * It is mainly useful for sink connectors, since the topic field is often used to determine the equivalent entity name in the destination system - * (e.g. database table or search index name). - */ public class TimestampRouter> implements Transformation { - public interface Keys { + public static final String OVERVIEW_DOC = + "Update the record's topic field as a function of the original topic value and the record timestamp." + + "

      " + + "This is mainly useful for sink connectors, since the topic field is often used to determine the equivalent entity name in the destination system" + + "(e.g. database table or search index name)."; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(ConfigName.TOPIC_FORMAT, ConfigDef.Type.STRING, "${topic}-${timestamp}", ConfigDef.Importance.HIGH, + "Format string which can contain ${topic} and ${timestamp} as placeholders for the topic and timestamp, respectively.") + .define(ConfigName.TIMESTAMP_FORMAT, ConfigDef.Type.STRING, "yyyyMMdd", ConfigDef.Importance.HIGH, + "Format string for the timestamp that is compatible with java.text.SimpleDateFormat."); + + private interface ConfigName { String TOPIC_FORMAT = "topic.format"; String TIMESTAMP_FORMAT = "timestamp.format"; } - private static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(Keys.TOPIC_FORMAT, ConfigDef.Type.STRING, "${topic}-${timestamp}", ConfigDef.Importance.HIGH, - "Format string which can contain ``${topic}`` and ``${timestamp}`` as placeholders for the topic and timestamp, respectively.") - .define(Keys.TIMESTAMP_FORMAT, ConfigDef.Type.STRING, "yyyyMMdd", ConfigDef.Importance.HIGH, - "Format string for the timestamp that is compatible with java.text.SimpleDateFormat."); - private String topicFormat; private ThreadLocal timestampFormat; @@ -53,9 +53,9 @@ public interface Keys { public void configure(Map props) { final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); - topicFormat = config.getString(Keys.TOPIC_FORMAT); + topicFormat = config.getString(ConfigName.TOPIC_FORMAT); - final String timestampFormatStr = config.getString(Keys.TIMESTAMP_FORMAT); + final String timestampFormatStr = config.getString(ConfigName.TIMESTAMP_FORMAT); timestampFormat = new ThreadLocal() { @Override protected SimpleDateFormat initialValue() { diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java new file mode 100644 index 0000000000000..504da541f6574 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java @@ -0,0 +1,111 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.transforms; + +import org.apache.kafka.common.cache.Cache; +import org.apache.kafka.common.cache.LRUCache; +import org.apache.kafka.common.cache.SynchronizedCache; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.kafka.connect.transforms.util.Requirements.requireMap; +import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct; + +public class ValueToKey> implements Transformation { + + public static final String OVERVIEW_DOC = "Replace the record key with a new key formed from a subset of fields in the record value."; + + public static final String FIELDS_CONFIG = "fields"; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, new NonEmptyListValidator(), ConfigDef.Importance.HIGH, + "Field names on the record value to extract as the record key."); + + private static final String PURPOSE = "copying fields from value to key"; + + private List fields; + + private Cache valueToKeySchemaCache; + + @Override + public void configure(Map configs) { + final SimpleConfig config = new SimpleConfig(CONFIG_DEF, configs); + fields = config.getList(FIELDS_CONFIG); + valueToKeySchemaCache = new SynchronizedCache<>(new LRUCache(16)); + } + + @Override + public R apply(R record) { + if (record.valueSchema() == null) { + return applySchemaless(record); + } else { + return applyWithSchema(record); + } + } + + private R applySchemaless(R record) { + final Map value = requireMap(record.value(), PURPOSE); + final Map key = new HashMap<>(fields.size()); + for (String field : fields) { + key.put(field, value.get(field)); + } + return record.newRecord(record.topic(), record.kafkaPartition(), null, key, record.valueSchema(), record.value(), record.timestamp()); + } + + private R applyWithSchema(R record) { + final Struct value = requireStruct(record.value(), PURPOSE); + + Schema keySchema = valueToKeySchemaCache.get(value.schema()); + if (keySchema == null) { + final SchemaBuilder keySchemaBuilder = SchemaBuilder.struct(); + for (String field : fields) { + final Schema fieldSchema = value.schema().field(field).schema(); + keySchemaBuilder.field(field, fieldSchema); + } + keySchema = keySchemaBuilder.build(); + valueToKeySchemaCache.put(value.schema(), keySchema); + } + + final Struct key = new Struct(keySchema); + for (String field : fields) { + key.put(field, value.get(field)); + } + + return record.newRecord(record.topic(), record.kafkaPartition(), keySchema, key, value.schema(), value, record.timestamp()); + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() { + valueToKeySchemaCache = null; + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java new file mode 100644 index 0000000000000..1abbbc8aaf081 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.transforms.util; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; + +import java.util.List; + +public class NonEmptyListValidator implements ConfigDef.Validator { + + @Override + public void ensureValid(String name, Object value) { + if (((List) value).isEmpty()) { + throw new ConfigException(name, value, "Empty list"); + } + } + + @Override + public String toString() { + return "non-empty list"; + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/RegexValidator.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/RegexValidator.java new file mode 100644 index 0000000000000..9713b27446f47 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/RegexValidator.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.kafka.connect.transforms.util; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; + +import java.util.regex.Pattern; + +public class RegexValidator implements ConfigDef.Validator { + + @Override + public void ensureValid(String name, Object value) { + try { + Pattern.compile((String) value); + } catch (Exception e) { + throw new ConfigException(name, value, "Invalid regex: " + e.getMessage()); + } + } + + @Override + public String toString() { + return "valid regex"; + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java new file mode 100644 index 0000000000000..b004f8ae91500 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.transforms.util; + +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.sink.SinkRecord; + +import java.util.Map; + +public class Requirements { + + public static void requireSchema(Schema schema, String purpose) { + if (schema == null) { + throw new DataException("Schema required for [" + purpose + "]"); + } + } + + public static Map requireMap(Object value, String purpose) { + if (!(value instanceof Map)) { + throw new DataException("Only Map objects supported in absence of schema for [" + purpose + "], found: " + nullSafeClassName(value)); + } + return (Map) value; + } + + public static Struct requireStruct(Object value, String purpose) { + if (!(value instanceof Struct)) { + throw new DataException("Only Struct objects supported for [" + purpose + "], found: " + nullSafeClassName(value)); + } + return (Struct) value; + } + + public static SinkRecord requireSinkRecord(ConnectRecord record, String purpose) { + if (!(record instanceof SinkRecord)) { + throw new DataException("Only SinkRecord supported for [" + purpose + "], found: " + nullSafeClassName(record)); + } + return (SinkRecord) record; + } + + private static String nullSafeClassName(Object x) { + return x == null ? "null" : x.getClass().getCanonicalName(); + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java new file mode 100644 index 0000000000000..da261e79b22de --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.transforms.util; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; + +import java.util.Map; + +public class SchemaUtil { + + public static SchemaBuilder copySchemaBasics(Schema source, SchemaBuilder builder) { + builder.name(source.name()); + builder.version(source.version()); + builder.doc(source.doc()); + + final Map params = source.parameters(); + if (params != null) { + builder.parameters(params); + } + + return builder; + } + +} diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ExtractFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ExtractFieldTest.java new file mode 100644 index 0000000000000..d72179559a50a --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ExtractFieldTest.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.transforms; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Test; + +import java.util.Collections; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +public class ExtractFieldTest { + + @Test + public void schemaless() { + final ExtractField xform = new ExtractField.Key<>(); + xform.configure(Collections.singletonMap("field", "magic")); + + final SinkRecord record = new SinkRecord("test", 0, null, Collections.singletonMap("magic", 42), null, null, 0); + final SinkRecord transformedRecord = xform.apply(record); + + assertNull(transformedRecord.keySchema()); + assertEquals(42, transformedRecord.key()); + } + + @Test + public void withSchema() { + final ExtractField xform = new ExtractField.Key<>(); + xform.configure(Collections.singletonMap("field", "magic")); + + final Schema keySchema = SchemaBuilder.struct().field("magic", Schema.INT32_SCHEMA).build(); + final Struct key = new Struct(keySchema).put("magic", 42); + final SinkRecord record = new SinkRecord("test", 0, keySchema, key, null, null, 0); + final SinkRecord transformedRecord = xform.apply(record); + + assertEquals(Schema.INT32_SCHEMA, transformedRecord.keySchema()); + assertEquals(42, transformedRecord.key()); + } + +} diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistToStructTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistFieldTest.java similarity index 71% rename from connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistToStructTest.java rename to connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistFieldTest.java index 99a6e99dde000..b5f9d93d935e4 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistToStructTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistFieldTest.java @@ -25,12 +25,25 @@ import java.util.Collections; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; -public class HoistToStructTest { +public class HoistFieldTest { @Test - public void sanityCheck() { - final HoistToStruct xform = new HoistToStruct.Key<>(); + public void schemaless() { + final HoistField xform = new HoistField.Key<>(); + xform.configure(Collections.singletonMap("field", "magic")); + + final SinkRecord record = new SinkRecord("test", 0, null, 42, null, null, 0); + final SinkRecord transformedRecord = xform.apply(record); + + assertNull(transformedRecord.keySchema()); + assertEquals(Collections.singletonMap("magic", 42), transformedRecord.key()); + } + + @Test + public void withSchema() { + final HoistField xform = new HoistField.Key<>(); xform.configure(Collections.singletonMap("field", "magic")); final SinkRecord record = new SinkRecord("test", 0, Schema.INT32_SCHEMA, 42, null, null, 0); diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java new file mode 100644 index 0000000000000..c96058afbe6de --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java @@ -0,0 +1,156 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.transforms; + +import org.apache.kafka.connect.data.Decimal; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.data.Time; +import org.apache.kafka.connect.data.Timestamp; +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Test; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class MaskFieldTest { + + private static MaskField transform(List fields) { + final MaskField xform = new MaskField.Value<>(); + xform.configure(Collections.singletonMap("fields", fields)); + return xform; + } + + private static SinkRecord record(Schema schema, Object value) { + return new SinkRecord("", 0, null, null, schema, value, 0); + } + + @Test + public void schemaless() { + final Map value = new HashMap<>(); + value.put("magic", 42); + value.put("bool", true); + value.put("byte", (byte) 42); + value.put("short", (short) 42); + value.put("int", 42); + value.put("long", 42L); + value.put("float", 42f); + value.put("double", 42d); + value.put("string", "blabla"); + value.put("date", new Date()); + value.put("bigint", new BigInteger("42")); + value.put("bigdec", new BigDecimal("42.0")); + value.put("list", Collections.singletonList(42)); + value.put("map", Collections.singletonMap("key", "value")); + + final List maskFields = new ArrayList<>(value.keySet()); + maskFields.remove("magic"); + + final Map updatedValue = (Map) transform(maskFields).apply(record(null, value)).value(); + + assertEquals(42, updatedValue.get("magic")); + assertEquals(false, updatedValue.get("bool")); + assertEquals((byte) 0, updatedValue.get("byte")); + assertEquals((short) 0, updatedValue.get("short")); + assertEquals(0, updatedValue.get("int")); + assertEquals(0L, updatedValue.get("long")); + assertEquals(0f, updatedValue.get("float")); + assertEquals(0d, updatedValue.get("double")); + assertEquals("", updatedValue.get("string")); + assertEquals(new Date(0), updatedValue.get("date")); + assertEquals(BigInteger.ZERO, updatedValue.get("bigint")); + assertEquals(BigDecimal.ZERO, updatedValue.get("bigdec")); + assertEquals(Collections.emptyList(), updatedValue.get("list")); + assertEquals(Collections.emptyMap(), updatedValue.get("map")); + } + + @Test + public void withSchema() { + Schema schema = SchemaBuilder.struct() + .field("magic", Schema.INT32_SCHEMA) + .field("bool", Schema.BOOLEAN_SCHEMA) + .field("byte", Schema.INT8_SCHEMA) + .field("short", Schema.INT16_SCHEMA) + .field("int", Schema.INT32_SCHEMA) + .field("long", Schema.INT64_SCHEMA) + .field("float", Schema.FLOAT32_SCHEMA) + .field("double", Schema.FLOAT64_SCHEMA) + .field("string", Schema.STRING_SCHEMA) + .field("date", org.apache.kafka.connect.data.Date.SCHEMA) + .field("time", Time.SCHEMA) + .field("timestamp", Timestamp.SCHEMA) + .field("decimal", Decimal.schema(0)) + .field("array", SchemaBuilder.array(Schema.INT32_SCHEMA)) + .field("map", SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA)) + .build(); + + final Struct value = new Struct(schema); + value.put("magic", 42); + value.put("bool", true); + value.put("byte", (byte) 42); + value.put("short", (short) 42); + value.put("int", 42); + value.put("long", 42L); + value.put("float", 42f); + value.put("double", 42d); + value.put("string", "hmm"); + value.put("date", new Date()); + value.put("time", new Date()); + value.put("timestamp", new Date()); + value.put("decimal", new BigDecimal(42)); + value.put("array", Arrays.asList(1, 2, 3)); + value.put("map", Collections.singletonMap("what", "what")); + + final List maskFields = new ArrayList<>(schema.fields().size()); + for (Field field: schema.fields()) { + if (!field.name().equals("magic")) { + maskFields.add(field.name()); + } + } + + final Struct updatedValue = (Struct) transform(maskFields).apply(record(schema, value)).value(); + + assertEquals(42, updatedValue.get("magic")); + assertEquals(false, updatedValue.get("bool")); + assertEquals((byte) 0, updatedValue.get("byte")); + assertEquals((short) 0, updatedValue.get("short")); + assertEquals(0, updatedValue.get("int")); + assertEquals(0L, updatedValue.get("long")); + assertEquals(0f, updatedValue.get("float")); + assertEquals(0d, updatedValue.get("double")); + assertEquals("", updatedValue.get("string")); + assertEquals(new Date(0), updatedValue.get("date")); + assertEquals(new Date(0), updatedValue.get("time")); + assertEquals(new Date(0), updatedValue.get("timestamp")); + assertEquals(BigDecimal.ZERO, updatedValue.get("decimal")); + assertEquals(Collections.emptyList(), updatedValue.get("array")); + assertEquals(Collections.emptyMap(), updatedValue.get("map")); + } + +} diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/RegexRouterTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/RegexRouterTest.java new file mode 100644 index 0000000000000..c599265d57200 --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/RegexRouterTest.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.transforms; + +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class RegexRouterTest { + + private static String apply(String regex, String replacement, String topic) { + final Map props = new HashMap<>(); + props.put("regex", regex); + props.put("replacement", replacement); + final RegexRouter router = new RegexRouter<>(); + router.configure(props); + return router.apply(new SinkRecord(topic, 0, null, null, null, null, 0)) + .topic(); + } + + @Test + public void staticReplacement() { + assertEquals("bar", apply("foo", "bar", "foo")); + } + + @Test + public void doesntMatch() { + assertEquals("orig", apply("foo", "bar", "orig")); + } + + @Test + public void identity() { + assertEquals("orig", apply("(.*)", "$1", "orig")); + } + + @Test + public void addPrefix() { + assertEquals("prefix-orig", apply("(.*)", "prefix-$1", "orig")); + } + + @Test + public void addSuffix() { + assertEquals("orig-suffix", apply("(.*)", "$1-suffix", "orig")); + } + + @Test + public void slice() { + assertEquals("index", apply("(.*)-(\\d\\d\\d\\d\\d\\d\\d\\d)", "$1", "index-20160117")); + } + +} diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java new file mode 100644 index 0000000000000..9f9d4b7031b52 --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java @@ -0,0 +1,92 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.transforms; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class ReplaceFieldTest { + + @Test + public void schemaless() { + final ReplaceField xform = new ReplaceField.Value<>(); + + final Map props = new HashMap<>(); + props.put("blacklist", "dont"); + props.put("renames", "abc:xyz,foo:bar"); + + xform.configure(props); + + final Map value = new HashMap<>(); + value.put("dont", "whatever"); + value.put("abc", 42); + value.put("foo", true); + value.put("etc", "etc"); + + final SinkRecord record = new SinkRecord("test", 0, null, null, null, value, 0); + final SinkRecord transformedRecord = xform.apply(record); + + final Map updatedValue = (Map) transformedRecord.value(); + assertEquals(3, updatedValue.size()); + assertEquals(42, updatedValue.get("xyz")); + assertEquals(true, updatedValue.get("bar")); + assertEquals("etc", updatedValue.get("etc")); + } + + @Test + public void withSchema() { + final ReplaceField xform = new ReplaceField.Value<>(); + + final Map props = new HashMap<>(); + props.put("whitelist", "abc,foo"); + props.put("renames", "abc:xyz,foo:bar"); + + xform.configure(props); + + final Schema schema = SchemaBuilder.struct() + .field("dont", Schema.STRING_SCHEMA) + .field("abc", Schema.INT32_SCHEMA) + .field("foo", Schema.BOOLEAN_SCHEMA) + .field("etc", Schema.STRING_SCHEMA) + .build(); + + final Struct value = new Struct(schema); + value.put("dont", "whatever"); + value.put("abc", 42); + value.put("foo", true); + value.put("etc", "etc"); + + final SinkRecord record = new SinkRecord("test", 0, null, null, schema, value, 0); + final SinkRecord transformedRecord = xform.apply(record); + + final Struct updatedValue = (Struct) transformedRecord.value(); + + assertEquals(2, updatedValue.schema().fields().size()); + assertEquals(new Integer(42), updatedValue.getInt32("xyz")); + assertEquals(true, updatedValue.getBoolean("bar")); + } + +} diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java new file mode 100644 index 0000000000000..2aa790f0a2d82 --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.transforms; + +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class SetSchemaMetadataTest { + + @Test + public void schemaNameUpdate() { + final SetSchemaMetadata xform = new SetSchemaMetadata.Value<>(); + xform.configure(Collections.singletonMap("schema.name", "foo")); + final SinkRecord record = new SinkRecord("", 0, null, null, SchemaBuilder.struct().build(), null, 0); + final SinkRecord updatedRecord = xform.apply(record); + assertEquals("foo", updatedRecord.valueSchema().name()); + } + + @Test + public void schemaVersionUpdate() { + final SetSchemaMetadata xform = new SetSchemaMetadata.Value<>(); + xform.configure(Collections.singletonMap("schema.version", 42)); + final SinkRecord record = new SinkRecord("", 0, null, null, SchemaBuilder.struct().build(), null, 0); + final SinkRecord updatedRecord = xform.apply(record); + assertEquals(new Integer(42), updatedRecord.valueSchema().version()); + } + + @Test + public void schemaNameAndVersionUpdate() { + final Map props = new HashMap<>(); + props.put("schema.name", "foo"); + props.put("schema.version", "42"); + + final SetSchemaMetadata xform = new SetSchemaMetadata.Value<>(); + xform.configure(props); + + final SinkRecord record = new SinkRecord("", 0, null, null, SchemaBuilder.struct().build(), null, 0); + + final SinkRecord updatedRecord = xform.apply(record); + + assertEquals("foo", updatedRecord.valueSchema().name()); + assertEquals(new Integer(42), updatedRecord.valueSchema().version()); + } + +} diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java new file mode 100644 index 0000000000000..e5328d36baebb --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java @@ -0,0 +1,87 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + **/ + +package org.apache.kafka.connect.transforms; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +public class ValueToKeyTest { + + @Test + public void schemaless() { + final ValueToKey xform = new ValueToKey<>(); + xform.configure(Collections.singletonMap("fields", "a,b")); + + final HashMap value = new HashMap<>(); + value.put("a", 1); + value.put("b", 2); + value.put("c", 3); + + final SinkRecord record = new SinkRecord("", 0, null, null, null, value, 0); + final SinkRecord transformedRecord = xform.apply(record); + + final HashMap expectedKey = new HashMap<>(); + expectedKey.put("a", 1); + expectedKey.put("b", 2); + + assertNull(transformedRecord.keySchema()); + assertEquals(expectedKey, transformedRecord.key()); + } + + @Test + public void withSchema() { + final ValueToKey xform = new ValueToKey<>(); + xform.configure(Collections.singletonMap("fields", "a,b")); + + final Schema valueSchema = SchemaBuilder.struct() + .field("a", Schema.INT32_SCHEMA) + .field("b", Schema.INT32_SCHEMA) + .field("c", Schema.INT32_SCHEMA) + .build(); + + final Struct value = new Struct(valueSchema); + value.put("a", 1); + value.put("b", 2); + value.put("c", 3); + + final SinkRecord record = new SinkRecord("", 0, null, null, valueSchema, value, 0); + final SinkRecord transformedRecord = xform.apply(record); + + final Schema expectedKeySchema = SchemaBuilder.struct() + .field("a", Schema.INT32_SCHEMA) + .field("b", Schema.INT32_SCHEMA) + .build(); + + final Struct expectedKey = new Struct(expectedKeySchema) + .put("a", 1) + .put("b", 2); + + assertEquals(expectedKeySchema, transformedRecord.keySchema()); + assertEquals(expectedKey, transformedRecord.key()); + } + +} diff --git a/docs/connect.html b/docs/connect.html index 23e168cae0309..1af5ed95f6051 100644 --- a/docs/connect.html +++ b/docs/connect.html @@ -100,6 +100,22 @@

      Configuring Connecto For any other options, you should consult the documentation for the connector. +

      Transformations

      + + Connectors can be configured with transformations to make lightweight message-at-a-time modifications. They can be convenient for minor data massaging and routing changes. + + A transformation chain can be specified in the connector configuration. + +
        +
      • transforms - List of aliases for the transformation, specifying the order in which the transformations will be applied.
      • +
      • transforms.$alias.type - Fully qualified class name for the transformation.
      • +
      • transforms.$alias.$transformationSpecificConfig Configuration properties for the transformation
      • +
      + + Several widely-applicable data and routing transformations are included with Kafka Connect: + + +

      REST API

      Since Kafka Connect is intended to be run as a service, it also provides a REST API for managing connectors. By default, this service runs on port 8083. The following are the currently supported endpoints: diff --git a/tests/kafkatest/tests/connect/connect_distributed_test.py b/tests/kafkatest/tests/connect/connect_distributed_test.py index 198e94501f50d..c298fb154e86e 100644 --- a/tests/kafkatest/tests/connect/connect_distributed_test.py +++ b/tests/kafkatest/tests/connect/connect_distributed_test.py @@ -460,7 +460,7 @@ def test_transformations(self): 'file': self.INPUT_FILE, 'topic': self.TOPIC, 'transforms': 'hoistToStruct,insertTimestampField', - 'transforms.hoistToStruct.type': 'org.apache.kafka.connect.transforms.HoistToStruct$Value', + 'transforms.hoistToStruct.type': 'org.apache.kafka.connect.transforms.HoistField$Value', 'transforms.hoistToStruct.field': 'content', 'transforms.insertTimestampField.type': 'org.apache.kafka.connect.transforms.InsertField$Value', 'transforms.insertTimestampField.timestamp.field': ts_fieldname, From db57c1a0c7e057e19ec165d3cf9fa35d54b97c8f Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Sat, 21 Jan 2017 13:18:39 -0800 Subject: [PATCH 022/177] KAFKA-4060 and KAFKA-4476 follow up: remove unnecessary InternalTopicManager branch and fixed one copartitioning bug ZK removed reveal a bug in `StreamPartitionAssigner` but did not fix it properly. This is a follow up bug fix. Issue: - If topic metadata is missing, `StreamPartitionAssigner` should not create any affected tasks that consume topics with missing metadata. - Depending downstream tasks should not be create either. - For tasks that are not created, no store changelog topics (if any) should get created - For tasks that write output to not-yet existing internal repartitioning topics, those repartitioning topics should not get created Author: Matthias J. Sax Reviewers: Damian Guy, Guozhang Wang Closes #2404 from mjsax/kafka-4060-zk-test-follow-up (cherry picked from commit 0b99bea590842018e8e97e7fd1c71b1471db4d08) Signed-off-by: Guozhang Wang --- .../processor/DefaultPartitionGrouper.java | 2 + .../internals/StreamPartitionAssignor.java | 59 ++++------ .../StreamPartitionAssignorTest.java | 103 +++++++++++------- 3 files changed, 90 insertions(+), 74 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java b/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java index 1da12097b53eb..2f354d8d3bc69 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.processor.internals.StreamPartitionAssignor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -83,6 +84,7 @@ protected int maxNumPartitions(Cluster metadata, Set topics) { if (partitions == null) { log.info("Skipping assigning topic {} to tasks since its metadata is not available yet", topic); + maxNumPartitions = StreamPartitionAssignor.NOT_AVAILABLE; } else { int numPartitions = partitions.size(); if (numPartitions > maxNumPartitions) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java index 7b48a6fdb784c..382c4e9986aeb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java @@ -59,7 +59,7 @@ public class StreamPartitionAssignor implements PartitionAssignor, Configurable private static final Logger log = LoggerFactory.getLogger(StreamPartitionAssignor.class); - public final static int UNKNOWN = -1; + private final static int UNKNOWN = -1; public final static int NOT_AVAILABLE = -2; private static class AssignedPartition implements Comparable { @@ -159,7 +159,7 @@ public int compare(TopicPartition p1, TopicPartition p2) { private Map> standbyTasks; private Map> activeTasks; - InternalTopicManager internalTopicManager; + private InternalTopicManager internalTopicManager; /** * We need to have the PartitionAssignor and its StreamThread to be mutually accessible @@ -369,9 +369,7 @@ public Map assign(Cluster metadata, Map topicPartitions) { + private void prepareTopic(final Map topicPartitions) { log.debug("stream-thread [{}] Starting to validate internal topics in partition assignor.", streamThread.getName()); - // if ZK is specified, prepare the internal source topic before calling partition grouper - if (internalTopicManager != null) { - for (Map.Entry entry : topicPartitions.entrySet()) { - InternalTopicConfig topic = entry.getValue().config; - Integer numPartitions = entry.getValue().numPartitions; + for (final Map.Entry entry : topicPartitions.entrySet()) { + final InternalTopicConfig topic = entry.getValue().config; + final Integer numPartitions = entry.getValue().numPartitions; - if (numPartitions == NOT_AVAILABLE) { - continue; - } - if (numPartitions < 0) { - throw new TopologyBuilderException(String.format("stream-thread [%s] Topic [%s] number of partitions not defined", streamThread.getName(), topic.name())); - } - - internalTopicManager.makeReady(topic, numPartitions); - - // wait until the topic metadata has been propagated to all brokers - List partitions; - do { - partitions = streamThread.restoreConsumer.partitionsFor(topic.name()); - } while (partitions == null || partitions.size() != numPartitions); + if (numPartitions == NOT_AVAILABLE) { + continue; } - } else { - List missingTopics = new ArrayList<>(); - for (String topic : topicPartitions.keySet()) { - List partitions = streamThread.restoreConsumer.partitionsFor(topic); - if (partitions == null) { - missingTopics.add(topic); - } + if (numPartitions < 0) { + throw new TopologyBuilderException(String.format("stream-thread [%s] Topic [%s] number of partitions not defined", streamThread.getName(), topic.name())); } - if (!missingTopics.isEmpty()) { - log.warn("stream-thread [{}] Topic {} do not exists but couldn't created as the config '{}' isn't supplied", - streamThread.getName(), missingTopics, StreamsConfig.ZOOKEEPER_CONNECT_CONFIG); - } + internalTopicManager.makeReady(topic, numPartitions); + + // wait until the topic metadata has been propagated to all brokers + List partitions; + do { + partitions = streamThread.restoreConsumer.partitionsFor(topic.name()); + } while (partitions == null || partitions.size() != numPartitions); } log.info("stream-thread [{}] Completed validating internal topics in partition assignor", streamThread.getName()); @@ -661,6 +643,11 @@ private void ensureCopartitioning(Set copartitionGroup, Arrays.sort(topics); throw new TopologyBuilderException(String.format("stream-thread [%s] Topics not co-partitioned: [%s]", streamThread.getName(), Utils.mkString(Arrays.asList(topics), ","))); } + } else { + if (allRepartitionTopicsNumPartitions.get(topic).numPartitions == NOT_AVAILABLE) { + numPartitions = NOT_AVAILABLE; + break; + } } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java index c212f1412d6d7..eff2179abcb5e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; @@ -38,23 +39,23 @@ import org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo; import org.apache.kafka.streams.state.HostInfo; import org.apache.kafka.test.MockClientSupplier; +import org.apache.kafka.test.MockInternalTopicManager; import org.apache.kafka.test.MockProcessorSupplier; -import org.apache.kafka.test.MockTimestampExtractor; import org.apache.kafka.test.MockStateStoreSupplier; -import org.apache.kafka.test.MockInternalTopicManager; +import org.apache.kafka.test.MockTimestampExtractor; import org.junit.Assert; import org.junit.Test; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; -import java.util.Set; -import java.util.Properties; import java.util.Map; +import java.util.Properties; +import java.util.Set; import java.util.UUID; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Collections; -import java.util.ArrayList; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; @@ -91,7 +92,7 @@ public class StreamPartitionAssignorTest { new PartitionInfo("topic3", 3, Node.noNode(), new Node[0], new Node[0]) ); - private Cluster metadata = new Cluster("cluster", Arrays.asList(Node.noNode()), infos, Collections.emptySet(), + private Cluster metadata = new Cluster("cluster", Collections.singletonList(Node.noNode()), infos, Collections.emptySet(), Collections.emptySet()); private final TaskId task0 = new TaskId(0, 0); @@ -248,17 +249,17 @@ public void testAssignWithPartialTopology() throws Exception { UUID uuid1 = UUID.randomUUID(); String client1 = "client1"; - StreamThread thread10 = new StreamThread(builder, config, new MockClientSupplier(), "test", client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); + final MockClientSupplier mockClientSupplier = new MockClientSupplier(); + StreamThread thread10 = new StreamThread(builder, config, mockClientSupplier, "test", client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread10, "test", client1)); + partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(thread10.config, mockClientSupplier.restoreConsumer)); Map subscriptions = new HashMap<>(); subscriptions.put("consumer10", new PartitionAssignor.Subscription(topics, new SubscriptionInfo(uuid1, Collections.emptySet(), Collections.emptySet(), userEndPoint).encode())); - // TODO: Update the code accordingly, - // This line was added to fix the test failure since internalTopicManager is created in the config method all the time. - partitionAssignor.internalTopicManager = null; + // will throw exception if it fails Map assignments = partitionAssignor.assign(metadata, subscriptions); @@ -285,7 +286,7 @@ public void testAssignEmptyMetadata() throws Exception { final Set prevTasks10 = Utils.mkSet(task0); final Set standbyTasks10 = Utils.mkSet(task1); - final Cluster emptyMetadata = new Cluster("cluster", Arrays.asList(Node.noNode()), + final Cluster emptyMetadata = new Cluster("cluster", Collections.singletonList(Node.noNode()), Collections.emptySet(), Collections.emptySet(), Collections.emptySet()); @@ -443,9 +444,6 @@ public void testAssignWithStates() throws Exception { subscriptions.put("consumer20", new PartitionAssignor.Subscription(topics, new SubscriptionInfo(uuid2, Collections.emptySet(), Collections.emptySet(), userEndPoint).encode())); - // TODO: Update the code accordingly, - // This line was added to fix the test failure since internalTopicManager is created in the config method all the time. - partitionAssignor.internalTopicManager = null; Map assignments = partitionAssignor.assign(metadata, subscriptions); // check assigned partition size: since there is no previous task and there are two sub-topologies the assignment is random so we cannot check exact match @@ -633,7 +631,7 @@ public void testAssignWithInternalTopics() throws Exception { partitionAssignor.setInternalTopicManager(internalTopicManager); Map subscriptions = new HashMap<>(); - Set emptyTasks = Collections.emptySet(); + Set emptyTasks = Collections.emptySet(); subscriptions.put("consumer10", new PartitionAssignor.Subscription(topics, new SubscriptionInfo(uuid1, emptyTasks, emptyTasks, userEndPoint).encode())); @@ -676,11 +674,11 @@ public void testAssignWithInternalTopicThatsSourceIsAnotherInternalTopic() throw partitionAssignor.setInternalTopicManager(internalTopicManager); Map subscriptions = new HashMap<>(); - Set emptyTasks = Collections.emptySet(); + Set emptyTasks = Collections.emptySet(); subscriptions.put("consumer10", new PartitionAssignor.Subscription(topics, new SubscriptionInfo(uuid1, emptyTasks, emptyTasks, userEndPoint).encode())); - Map assignments = partitionAssignor.assign(metadata, subscriptions); + partitionAssignor.assign(metadata, subscriptions); // check prepared internal topics assertEquals(2, internalTopicManager.readyTopics.size()); @@ -742,7 +740,7 @@ public void shouldMapUserEndPointToTopicPartitions() throws Exception { partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamThread.config, clientSupplier.restoreConsumer)); final Map subscriptions = new HashMap<>(); - final Set emptyTasks = Collections.emptySet(); + final Set emptyTasks = Collections.emptySet(); subscriptions.put("consumer1", new PartitionAssignor.Subscription(topics, new SubscriptionInfo(uuid1, emptyTasks, emptyTasks, myEndPoint).encode())); @@ -815,7 +813,7 @@ public void shouldThrowExceptionIfApplicationServerConfigPortIsNotAnInteger() th @Test public void shouldExposeHostStateToTopicPartitionsOnAssignment() throws Exception { final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); - List topic = Arrays.asList(new TopicPartition("topic", 0)); + List topic = Collections.singletonList(new TopicPartition("topic", 0)); final Map> hostState = Collections.singletonMap(new HostInfo("localhost", 80), Collections.singleton(new TopicPartition("topic", 0))); @@ -830,7 +828,7 @@ public void shouldExposeHostStateToTopicPartitionsOnAssignment() throws Exceptio public void shouldSetClusterMetadataOnAssignment() throws Exception { final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); - final List topic = Arrays.asList(new TopicPartition("topic", 0)); + final List topic = Collections.singletonList(new TopicPartition("topic", 0)); final Map> hostState = Collections.singletonMap(new HostInfo("localhost", 80), Collections.singleton(new TopicPartition("topic", 0))); @@ -863,22 +861,47 @@ public void shouldNotLoopInfinitelyOnMissingMetadataAndShouldNotCreateRelatedTas builder.setApplicationId(applicationId); KStream stream1 = builder + + // Task 1 (should get created): .stream("topic1") + // force repartitioning for aggregation .selectKey(new KeyValueMapper() { @Override public Object apply(Object key, Object value) { return null; } }) - .through("topic2"); + .groupByKey() + + // Task 2 (should get created): + // create repartioning and changelog topic as task 1 exists + .count("count") + + // force repartitioning for join, but second join input topic unknown + // -> internal repartitioning topic should not get created + .toStream() + .map(new KeyValueMapper>() { + @Override + public KeyValue apply(Object key, Long value) { + return null; + } + }); + builder + // Task 3 (should not get created because input topic unknown) .stream("unknownTopic") + + // force repartitioning for join, but input topic unknown + // -> thus should not create internal repartitioning topic .selectKey(new KeyValueMapper() { @Override public Object apply(Object key, Object value) { return null; } }) + + // Task 4 (should not get created because input topics unknown) + // should not create any of both input repartition topics or any of both changelog topics .join( stream1, new ValueJoiner() { @@ -894,13 +917,16 @@ public Object apply(Object value1, Object value2) { final String client = "client1"; final StreamsConfig config = new StreamsConfig(configProps()); - final StreamThread streamThread = new StreamThread(builder, config, new MockClientSupplier(), applicationId, client, uuid, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); + final MockClientSupplier mockClientSupplier = new MockClientSupplier(); + final StreamThread streamThread = new StreamThread(builder, config, mockClientSupplier, applicationId, client, uuid, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(streamThread, applicationId, client)); + final MockInternalTopicManager mockInternalTopicManager = new MockInternalTopicManager(streamThread.config, mockClientSupplier.restoreConsumer); + partitionAssignor.setInternalTopicManager(mockInternalTopicManager); final Map subscriptions = new HashMap<>(); - final Set emptyTasks = Collections.emptySet(); + final Set emptyTasks = Collections.emptySet(); subscriptions.put( client, new PartitionAssignor.Subscription( @@ -908,20 +934,22 @@ public Object apply(Object value1, Object value2) { new SubscriptionInfo(uuid, emptyTasks, emptyTasks, userEndPoint).encode() ) ); - // TODO: Update the code accordingly, - // This line was added to fix the test failure since internalTopicManager is created in the config method all the time. - partitionAssignor.internalTopicManager = null; final Map assignment = partitionAssignor.assign(metadata, subscriptions); + final Map expectedCreatedInternalTopics = new HashMap<>(); + expectedCreatedInternalTopics.put(applicationId + "-count-repartition", 3); + expectedCreatedInternalTopics.put(applicationId + "-count-changelog", 3); + assertThat(mockInternalTopicManager.readyTopics, equalTo(expectedCreatedInternalTopics)); + final List expectedAssignment = Arrays.asList( new TopicPartition("topic1", 0), new TopicPartition("topic1", 1), new TopicPartition("topic1", 2), - new TopicPartition("topic2", 0), - new TopicPartition("topic2", 1), - new TopicPartition("topic2", 2) + new TopicPartition(applicationId + "-count-repartition", 0), + new TopicPartition(applicationId + "-count-repartition", 1), + new TopicPartition(applicationId + "-count-repartition", 2) ); - assertThat(expectedAssignment, equalTo(assignment.get(client).partitions())); + assertThat(new HashSet(assignment.get(client).partitions()), equalTo(new HashSet(expectedAssignment))); } @Test @@ -981,13 +1009,15 @@ public void shouldNotAddStandbyTaskPartitionsToPartitionsForHost() throws Except final UUID uuid = UUID.randomUUID(); final String client = "client1"; - final StreamThread streamThread = new StreamThread(builder, config, new MockClientSupplier(), applicationId, client, uuid, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); + final MockClientSupplier mockClientSupplier = new MockClientSupplier(); + final StreamThread streamThread = new StreamThread(builder, config, mockClientSupplier, applicationId, client, uuid, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(streamThread, applicationId, client)); + partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamThread.config, mockClientSupplier.restoreConsumer)); final Map subscriptions = new HashMap<>(); - final Set emptyTasks = Collections.emptySet(); + final Set emptyTasks = Collections.emptySet(); subscriptions.put( "consumer1", new PartitionAssignor.Subscription( @@ -1004,9 +1034,6 @@ public void shouldNotAddStandbyTaskPartitionsToPartitionsForHost() throws Except ) ); final Set allPartitions = Utils.mkSet(t1p0, t1p1, t1p2); - // TODO: Update the code accordingly, - // This line was added to fix the test failure since internalTopicManager is created in the config method all the time. - partitionAssignor.internalTopicManager = null; final Map assign = partitionAssignor.assign(metadata, subscriptions); final PartitionAssignor.Assignment consumer1Assignment = assign.get("consumer1"); final AssignmentInfo assignmentInfo = AssignmentInfo.decode(consumer1Assignment.userData()); From e393c9928972c00677250ae2df3b4f061fdccd58 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Sat, 21 Jan 2017 14:10:01 -0800 Subject: [PATCH 023/177] KAFKA-4672; fix source compatibility for lambda expressions MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Variance changes introduced in KIP-100 cause compilation failures with lambda expression in Java 8. To my knowledge this only affects the following method `KStreams.transform(TransformerSupplier<...>, String...)` prior to the changes it was possible to write: `streams.transform(MyTransformer::new)` where `MyTransformer` extends `Transformer` After the changes the Java compiler is unable to infer correct return types for the lambda expressions. This change fixed this by reverting to invariant return types for transformer suppliers. please cherry-pick into 0.10.2.x Author: Xavier Léauté Reviewers: Ismael Juma, Damian Guy, Guozhang Wang Closes #2402 from xvrl/lambdas-oh-my (cherry picked from commit 20e957c9b1f389621bb8ae9e9f5554b66cd92a4f) Signed-off-by: Guozhang Wang --- .../src/main/java/org/apache/kafka/streams/kstream/KStream.java | 2 +- .../org/apache/kafka/streams/kstream/internals/KStreamImpl.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 3e95e7cec0d26..35095230cc3f8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -623,7 +623,7 @@ void to(final Serde keySerde, * @see #transformValues(ValueTransformerSupplier, String...) * @see #process(ProcessorSupplier, String...) */ - KStream transform(final TransformerSupplier> transformerSupplier, + KStream transform(final TransformerSupplier> transformerSupplier, final String... stateStoreNames); /** diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index b99e55d29d8e6..3d41ae44d768d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -365,7 +365,7 @@ public void to(Serde keySerde, Serde valSerde, StreamPartitioner KStream transform(TransformerSupplier> transformerSupplier, String... stateStoreNames) { + public KStream transform(TransformerSupplier> transformerSupplier, String... stateStoreNames) { Objects.requireNonNull(transformerSupplier, "transformerSupplier can't be null"); String name = topology.newName(TRANSFORM_NAME); From f0449d3248801facca442d93552a829e8b8d06a7 Mon Sep 17 00:00:00 2001 From: Eno Thereska Date: Mon, 23 Jan 2017 14:45:30 +0000 Subject: [PATCH 024/177] MINOR: Pass RecordingLevel to MetricConfig in the broker This is a KIP-104/105 follow-up. Thanks to ijuma for pointing out. Author: Eno Thereska Reviewers: Ismael Juma Closes #2350 from enothereska/minor-broker-level-config (cherry picked from commit 1eb1e2f60ae006144757fc9fc10ab423c58970bb) Signed-off-by: Ismael Juma --- .../main/scala/kafka/server/KafkaConfig.scala | 1 + .../main/scala/kafka/server/KafkaServer.scala | 12 +++-- .../unit/kafka/server/KafkaConfigTest.scala | 4 ++ .../unit/kafka/server/ServerMetricsTest.scala | 51 +++++++++++++++++++ 4 files changed, 64 insertions(+), 4 deletions(-) create mode 100755 core/src/test/scala/unit/kafka/server/ServerMetricsTest.scala diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 3c2a72d8ac6d9..c4f54c9e14a31 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -941,6 +941,7 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean) extends Abstra val metricNumSamples = getInt(KafkaConfig.MetricNumSamplesProp) val metricSampleWindowMs = getLong(KafkaConfig.MetricSampleWindowMsProp) val metricReporterClasses: java.util.List[MetricsReporter] = getConfiguredInstances(KafkaConfig.MetricReporterClassesProp, classOf[MetricsReporter]) + val metricRecordingLevel = getString(KafkaConfig.MetricRecordingLevelProp) /** ********* SSL Configuration **************/ val principalBuilderClass = getClass(KafkaConfig.PrincipalBuilderClassProp) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 79548e86c5969..b5075f9472884 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -79,9 +79,15 @@ object KafkaServer { logProps.put(LogConfig.MessageTimestampDifferenceMaxMsProp, kafkaConfig.logMessageTimestampDifferenceMaxMs) logProps } -} + private[server] def metricConfig(kafkaConfig: KafkaConfig): MetricConfig = { + new MetricConfig() + .samples(kafkaConfig.metricNumSamples) + .recordLevel(Sensor.RecordingLevel.forName(kafkaConfig.metricRecordingLevel)) + .timeWindow(kafkaConfig.metricSampleWindowMs, TimeUnit.MILLISECONDS) + } +} /** * Represents the lifecycle of a single Kafka broker. Handles all functionality required @@ -100,9 +106,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP var metrics: Metrics = null - private val metricConfig: MetricConfig = new MetricConfig() - .samples(config.metricNumSamples) - .timeWindow(config.metricSampleWindowMs, TimeUnit.MILLISECONDS) + private val metricConfig: MetricConfig = KafkaServer.metricConfig(config) val brokerState: BrokerState = new BrokerState diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 259178c57710b..0f5ff5dec6300 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -24,6 +24,7 @@ import kafka.cluster.EndPoint import kafka.message._ import kafka.utils.{CoreUtils, TestUtils} import org.apache.kafka.common.config.ConfigException +import org.apache.kafka.common.metrics.Sensor import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.SecurityProtocol import org.junit.Assert._ @@ -684,6 +685,8 @@ class KafkaConfigTest { //For LogFlushIntervalMsProp defaults.put(KafkaConfig.LogFlushSchedulerIntervalMsProp, "123") defaults.put(KafkaConfig.OffsetsTopicCompressionCodecProp, SnappyCompressionCodec.codec.toString) + // For MetricRecordingLevelProp + defaults.put(KafkaConfig.MetricRecordingLevelProp, Sensor.RecordingLevel.DEBUG.toString) val config = KafkaConfig.fromProps(defaults) assertEquals("127.0.0.1:2181", config.zkConnect) @@ -701,6 +704,7 @@ class KafkaConfigTest { assertEquals(10 * 60L * 1000L * 60, config.logRetentionTimeMillis) assertEquals(123L, config.logFlushIntervalMs) assertEquals(SnappyCompressionCodec, config.offsetsTopicCompressionCodec) + assertEquals(Sensor.RecordingLevel.DEBUG.toString, config.metricRecordingLevel) } private def assertPropertyInvalid(validRequiredProps: => Properties, name: String, values: Any*) { diff --git a/core/src/test/scala/unit/kafka/server/ServerMetricsTest.scala b/core/src/test/scala/unit/kafka/server/ServerMetricsTest.scala new file mode 100755 index 0000000000000..dc96680c95893 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ServerMetricsTest.scala @@ -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 kafka.server + +import kafka.utils.TestUtils +import org.apache.kafka.common.metrics.Sensor +import org.junit.Assert._ +import org.junit.Test +import org.scalatest.junit.JUnitSuite + +class ServerMetricsTest extends JUnitSuite { + + @Test + def testMetricsConfig(): Unit = { + val recordingLevels = List(Sensor.RecordingLevel.DEBUG, Sensor.RecordingLevel.INFO) + val illegalNames = List("IllegalName", "") + val props = TestUtils.createBrokerConfig(0, "localhost:2818") + + for (recordingLevel <- recordingLevels) { + props.put(KafkaConfig.MetricRecordingLevelProp, recordingLevel.name) + val config = KafkaConfig.fromProps(props) + val metricConfig = KafkaServer.metricConfig(config) + assertEquals(recordingLevel, metricConfig.recordLevel) + } + + for (illegalName <- illegalNames) { + intercept[IllegalArgumentException] { + props.put(KafkaConfig.MetricRecordingLevelProp, illegalName) + val config = KafkaConfig.fromProps(props) + KafkaServer.metricConfig(config) + } + } + + } + +} From afde87804211dfea2b9ee87c5f6470e8460dcd33 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Mon, 23 Jan 2017 09:34:52 -0800 Subject: [PATCH 025/177] HOTFIX: KAFKA-4060 and KAFKA-4476 follow up Author: Matthias J. Sax Reviewers: Damian Guy, Guozhang Wang Closes #2418 from mjsax/kafka-4060-zk-test-follow-up (cherry picked from commit dd897bdb2e9cc8790d1e0494fb7867a5cd09ecc6) Signed-off-by: Guozhang Wang --- .../processor/DefaultPartitionGrouper.java | 2 +- .../DefaultPartitionGrouperTest.java | 72 ++++++++++++------- 2 files changed, 49 insertions(+), 25 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java b/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java index 2f354d8d3bc69..25efcd6eaf391 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java @@ -84,7 +84,7 @@ protected int maxNumPartitions(Cluster metadata, Set topics) { if (partitions == null) { log.info("Skipping assigning topic {} to tasks since its metadata is not available yet", topic); - maxNumPartitions = StreamPartitionAssignor.NOT_AVAILABLE; + return StreamPartitionAssignor.NOT_AVAILABLE; } else { int numPartitions = partitions.size(); if (numPartitions > maxNumPartitions) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/DefaultPartitionGrouperTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/DefaultPartitionGrouperTest.java index e36bde4775876..e26453dc15846 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/DefaultPartitionGrouperTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/DefaultPartitionGrouperTest.java @@ -21,8 +21,6 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; - -import static org.apache.kafka.common.utils.Utils.mkSet; import org.junit.Test; import java.util.Arrays; @@ -32,11 +30,12 @@ import java.util.Map; import java.util.Set; +import static org.apache.kafka.common.utils.Utils.mkSet; import static org.junit.Assert.assertEquals; public class DefaultPartitionGrouperTest { - private List infos = Arrays.asList( + private final List infos = Arrays.asList( new PartitionInfo("topic1", 0, Node.noNode(), new Node[0], new Node[0]), new PartitionInfo("topic1", 1, Node.noNode(), new Node[0], new Node[0]), new PartitionInfo("topic1", 2, Node.noNode(), new Node[0], new Node[0]), @@ -44,39 +43,64 @@ public class DefaultPartitionGrouperTest { new PartitionInfo("topic2", 1, Node.noNode(), new Node[0], new Node[0]) ); - private Cluster metadata = new Cluster("cluster", Collections.singletonList(Node.noNode()), infos, Collections.emptySet(), + private final Cluster metadata = new Cluster( + "cluster", + Collections.singletonList(Node.noNode()), + infos, + Collections.emptySet(), Collections.emptySet()); @Test - public void testGrouping() { - PartitionGrouper grouper = new DefaultPartitionGrouper(); - int topicGroupId; - Map> expectedPartitionsForTask; - Map> topicGroups; - - topicGroups = new HashMap<>(); - topicGroups.put(0, mkSet("topic1")); - topicGroups.put(1, mkSet("topic2")); - - expectedPartitionsForTask = new HashMap<>(); - topicGroupId = 0; + public void shouldComputeGroupingForTwoGroups() { + final PartitionGrouper grouper = new DefaultPartitionGrouper(); + final Map> expectedPartitionsForTask = new HashMap<>(); + final Map> topicGroups = new HashMap<>(); + + int topicGroupId = 0; + + topicGroups.put(topicGroupId, mkSet("topic1")); expectedPartitionsForTask.put(new TaskId(topicGroupId, 0), mkSet(new TopicPartition("topic1", 0))); expectedPartitionsForTask.put(new TaskId(topicGroupId, 1), mkSet(new TopicPartition("topic1", 1))); expectedPartitionsForTask.put(new TaskId(topicGroupId, 2), mkSet(new TopicPartition("topic1", 2))); - topicGroupId++; + + topicGroups.put(++topicGroupId, mkSet("topic2")); expectedPartitionsForTask.put(new TaskId(topicGroupId, 0), mkSet(new TopicPartition("topic2", 0))); expectedPartitionsForTask.put(new TaskId(topicGroupId, 1), mkSet(new TopicPartition("topic2", 1))); assertEquals(expectedPartitionsForTask, grouper.partitionGroups(topicGroups, metadata)); + } - topicGroups = new HashMap<>(); - topicGroups.put(0, mkSet("topic1", "topic2")); + @Test + public void shouldComputeGroupingForSingleGroupWithMultipleTopics() { + final PartitionGrouper grouper = new DefaultPartitionGrouper(); + final Map> expectedPartitionsForTask = new HashMap<>(); + final Map> topicGroups = new HashMap<>(); - expectedPartitionsForTask = new HashMap<>(); - topicGroupId = 0; - expectedPartitionsForTask.put(new TaskId(topicGroupId, 0), mkSet(new TopicPartition("topic1", 0), new TopicPartition("topic2", 0))); - expectedPartitionsForTask.put(new TaskId(topicGroupId, 1), mkSet(new TopicPartition("topic1", 1), new TopicPartition("topic2", 1))); - expectedPartitionsForTask.put(new TaskId(topicGroupId, 2), mkSet(new TopicPartition("topic1", 2))); + final int topicGroupId = 0; + + topicGroups.put(topicGroupId, mkSet("topic1", "topic2")); + expectedPartitionsForTask.put( + new TaskId(topicGroupId, 0), + mkSet(new TopicPartition("topic1", 0), new TopicPartition("topic2", 0))); + expectedPartitionsForTask.put( + new TaskId(topicGroupId, 1), + mkSet(new TopicPartition("topic1", 1), new TopicPartition("topic2", 1))); + expectedPartitionsForTask.put( + new TaskId(topicGroupId, 2), + mkSet(new TopicPartition("topic1", 2))); + + assertEquals(expectedPartitionsForTask, grouper.partitionGroups(topicGroups, metadata)); + } + + @Test + public void shouldNotCreateAnyTasksBecauseOneTopicHasUnknownPartitions() { + final PartitionGrouper grouper = new DefaultPartitionGrouper(); + final Map> expectedPartitionsForTask = new HashMap<>(); + final Map> topicGroups = new HashMap<>(); + + final int topicGroupId = 0; + + topicGroups.put(topicGroupId, mkSet("topic1", "unknownTopic", "topic2")); assertEquals(expectedPartitionsForTask, grouper.partitionGroups(topicGroups, metadata)); } From 80ceb75d2acf5c3d8c21e03bf55e706c637d9fd9 Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Mon, 23 Jan 2017 10:21:06 -0800 Subject: [PATCH 026/177] KAFKA-4060 Follow-up: update docs accordingly Updated the docs with changes in KAFKA-4060. Author: Hojjat Jafarpour Author: Hojjat Jafarpour Reviewers: Ismael Juma, Matthias J. Sax, Guozhang Wang Closes #2377 from hjafarpour/KAFKA-4060-docs-update (cherry picked from commit 62206de4e52dbec32b99baffa0ebf7f5303d7b7f) Signed-off-by: Guozhang Wang --- docs/upgrade.html | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/upgrade.html b/docs/upgrade.html index a6683ba8b16cc..23b02afcfabb4 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -37,6 +37,8 @@
      Notable changes in A new close API with timeout has been added to KafkaConsumer to control the maximum wait time.
    12. Multiple regular expressions separated by commas can be passed to MirrorMaker with the new Java consumer via the --whitelist option. This makes the behaviour consistent with MirrorMaker when used the old Scala consumer.
    13. +
    14. Zookeeper dependency was removed from the Streams API. Now the Streams API uses Kafka protocol to manage internal topics instead of modifying Zookeeper directly. This eliminates the need for privilege[s] to access Zookeeper directly and "StreamsConfig.ZOOKEEPER_CONFIG" should not be set in the streams app anymore. If the Kafka cluster is secured, the Streams apps require to have proper security privilege[s] to create new topics.
    15. +
    16. Several new fields including "security.protocol", "connections.max.idle.ms", "retry.backoff.ms", "reconnect.backoff.ms" and "request.timeout.ms" were added to StreamsConfig class. User should pay attenntion to the default values and set these if needed. For more details please refer to 3.5 Kafka Streams Configs.
    17. Upgrading from 0.8.x, 0.9.x or 0.10.0.X to 0.10.1.0

      From 0f87991d5dbe8741953be04628810d76a9ee7434 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 23 Jan 2017 10:45:45 -0800 Subject: [PATCH 027/177] KAFKA-4633; Always using regex pattern subscription in StreamThread 1. In StreamThread, always use subscribe(Pattern, ..) function in order to avoid sending MetadataRequest with specific topic names and cause brokers to possibly auto-create subscribed topics; the pattern is generated as "topic-1|topic-2..|topic-n". 2. In ConsumerCoordinator, let the leader to refresh its metadata if the generated assignment contains some topics that is not contained in the subscribed topics; also in SubscriptionState, modified the verification for regex subscription to against the regex pattern instead of the matched topics since the returned assignment may contain some topics not yet created when joining the group but existed after the rebalance; also modified some unit tests in `KafkaConsumerTest` to accommodate the above changes. 3. Minor cleanup: changed String[] to List to avoid overloaded functions. 4. Minor cleanup: enforced strong typing in SinkNodeFactory and removed unnecessary unchecked tags. 5. Minor cleanup: augmented unit test error message and fixed a potential transient failure in KafkaStreamTest. Author: Guozhang Wang Reviewers: Damian Guy , Matthias J. Sax , Jason Gustafson Closes #2379 from guozhangwang/K4633-regex-pattern (cherry picked from commit 3400d0c3cc07d5d3136bf9a19142b36cca93f92d) Signed-off-by: Jason Gustafson --- .../internals/ConsumerCoordinator.java | 59 ++- .../consumer/internals/SubscriptionState.java | 13 +- .../clients/consumer/KafkaConsumerTest.java | 6 +- .../internals/ConsumerCoordinatorTest.java | 410 +++++++++++------- .../internals/SubscriptionStateTest.java | 53 ++- .../kstream/internals/AbstractStream.java | 6 - .../streams/processor/TopologyBuilder.java | 182 +++----- .../streams/processor/internals/SinkNode.java | 4 +- .../processor/internals/SourceNode.java | 9 +- .../processor/internals/StreamThread.java | 13 +- .../internals/StreamsMetadataState.java | 16 +- .../kafka/streams/KafkaStreamsTest.java | 2 + .../KStreamRepartitionJoinTest.java | 6 +- .../utils/EmbeddedKafkaCluster.java | 2 +- .../utils/IntegrationTestUtils.java | 4 +- .../streams/kstream/KStreamBuilderTest.java | 12 +- .../processor/TopologyBuilderTest.java | 21 +- .../SourceNodeRecordDeserializerTest.java | 4 +- .../org/apache/kafka/test/MockSourceNode.java | 3 +- 19 files changed, 478 insertions(+), 347 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index 7c463d10c9e8c..4c54a8f087b21 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -216,6 +216,32 @@ protected void onJoinComplete(int generation, // update partition assignment subscriptions.assignFromSubscribed(assignment.partitions()); + // check if the assignment contains some topics that were not in the original + // subscription, if yes we will obey what leader has decided and add these topics + // into the subscriptions as long as they still match the subscribed pattern + // + // TODO this part of the logic should be removed once we allow regex on leader assign + Set addedTopics = new HashSet<>(); + for (TopicPartition tp : subscriptions.assignedPartitions()) { + if (!joinedSubscription.contains(tp.topic())) + addedTopics.add(tp.topic()); + } + + if (!addedTopics.isEmpty()) { + Set newSubscription = new HashSet<>(subscriptions.subscription()); + Set newJoinedSubscription = new HashSet<>(joinedSubscription); + newSubscription.addAll(addedTopics); + newJoinedSubscription.addAll(addedTopics); + + this.subscriptions.subscribeFromPattern(newSubscription); + this.joinedSubscription = newJoinedSubscription; + } + + // update the metadata and enforce a refresh to make sure the fetcher can start + // fetching data in the next iteration + this.metadata.setTopics(subscriptions.groupSubscription()); + client.ensureFreshMetadata(); + // give the assignor a chance to update internal state based on the received assignment assignor.onAssignment(assignment); @@ -307,13 +333,44 @@ protected Map performAssignment(String leaderId, client.ensureFreshMetadata(); isLeader = true; - assignmentSnapshot = metadataSnapshot; log.debug("Performing assignment for group {} using strategy {} with subscriptions {}", groupId, assignor.name(), subscriptions); Map assignment = assignor.assign(metadata.fetch(), subscriptions); + // user-customized assignor may have created some topics that are not in the subscription list + // and assign their partitions to the members; in this case we would like to update the leader's + // own metadata with the newly added topics so that it will not trigger a subsequent rebalance + // when these topics gets updated from metadata refresh. + // + // TODO: this is a hack and not something we want to support long-term unless we push regex into the protocol + // we may need to modify the PartitionAssingor API to better support this case. + Set assignedTopics = new HashSet<>(); + for (Assignment assigned : assignment.values()) { + for (TopicPartition tp : assigned.partitions()) + assignedTopics.add(tp.topic()); + } + + if (!assignedTopics.containsAll(allSubscribedTopics)) { + Set notAssignedTopics = new HashSet<>(allSubscribedTopics); + notAssignedTopics.removeAll(assignedTopics); + log.warn("The following subscribed topics are not assigned to any members in the group {} : {} ", groupId, notAssignedTopics); + } + + if (!allSubscribedTopics.containsAll(assignedTopics)) { + Set newlyAddedTopics = new HashSet<>(assignedTopics); + newlyAddedTopics.removeAll(allSubscribedTopics); + log.info("The following not-subscribed topics are assigned to group {}, and their metadata will be fetched from the brokers : {}", groupId, newlyAddedTopics); + + allSubscribedTopics.addAll(assignedTopics); + this.subscriptions.groupSubscribe(allSubscribedTopics); + metadata.setTopics(this.subscriptions.groupSubscription()); + client.ensureFreshMetadata(); + } + + assignmentSnapshot = metadataSnapshot; + log.debug("Finished assignment for group {}: {}", groupId, assignment); Map groupAssignment = new HashMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index 9e496ff08ce5b..1a2a7ee6d76bd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -183,9 +183,16 @@ public void assignFromSubscribed(Collection assignments) { Set newAssignment = new HashSet<>(assignments); removeAllLagSensors(newAssignment); - for (TopicPartition tp : assignments) - if (!this.subscription.contains(tp.topic())) - throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic."); + if (this.subscribedPattern != null) { + for (TopicPartition tp : assignments) { + if (!this.subscribedPattern.matcher(tp.topic()).matches()) + throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic regex pattern; subscription pattern is " + this.subscribedPattern); + } + } else { + for (TopicPartition tp : assignments) + if (!this.subscription.contains(tp.topic())) + throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic; subscription is " + this.subscription); + } // after rebalancing, we always reinitialize the assignment value this.assignment.set(partitionToStateMap(assignments)); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 3e8310d410268..ac88ce9284420 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -366,7 +366,7 @@ public void verifyHeartbeatSent() throws Exception { Metadata metadata = new Metadata(0, Long.MAX_VALUE); metadata.update(cluster, time.milliseconds()); - MockClient client = new MockClient(time); + MockClient client = new MockClient(time, metadata); client.setNode(node); PartitionAssignor assignor = new RoundRobinAssignor(); @@ -1228,7 +1228,7 @@ private void consumerCloseTest(final long closeTimeoutMs, Metadata metadata = new Metadata(0, Long.MAX_VALUE); metadata.update(cluster, time.milliseconds()); - MockClient client = new MockClient(time); + MockClient client = new MockClient(time, metadata); client.setNode(node); PartitionAssignor assignor = new RoundRobinAssignor(); @@ -1238,6 +1238,8 @@ private void consumerCloseTest(final long closeTimeoutMs, consumer.subscribe(Arrays.asList(topic), getConsumerRebalanceListener(consumer)); Node coordinator = prepareRebalance(client, node, assignor, Arrays.asList(tp0), null); + client.prepareMetadataUpdate(cluster); + // Poll with responses client.prepareResponseFrom(fetchResponse(tp0, 0, 1), node); client.prepareResponseFrom(fetchResponse(tp0, 1, 0), node); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index ee6afe19de5ac..3c4dd2dac5deb 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -85,9 +85,11 @@ public class ConsumerCoordinatorTest { - private String topicName = "test"; + private String topic1 = "test1"; + private String topic2 = "test2"; private String groupId = "test-group"; - private TopicPartition tp = new TopicPartition(topicName, 0); + private TopicPartition t1p = new TopicPartition(topic1, 0); + private TopicPartition t2p = new TopicPartition(topic2, 0); private int rebalanceTimeoutMs = 60000; private int sessionTimeoutMs = 10000; private int heartbeatIntervalMs = 5000; @@ -98,7 +100,12 @@ public class ConsumerCoordinatorTest { private List assignors = Collections.singletonList(partitionAssignor); private MockTime time; private MockClient client; - private Cluster cluster = TestUtils.singletonCluster(topicName, 1); + private Cluster cluster = TestUtils.clusterWith(1, new HashMap() { + { + put(topic1, 1); + put(topic2, 1); + } + }); private Node node = cluster.nodes().get(0); private SubscriptionState subscriptions; private Metadata metadata; @@ -112,10 +119,10 @@ public class ConsumerCoordinatorTest { @Before public void setup() { this.time = new MockTime(); - this.client = new MockClient(time); this.subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST, metrics); this.metadata = new Metadata(0, Long.MAX_VALUE); this.metadata.update(cluster, time.milliseconds()); + this.client = new MockClient(time, metadata); this.consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000); this.metrics = new Metrics(time); this.rebalanceListener = new MockRebalanceListener(); @@ -157,7 +164,7 @@ public void testGroupDescribeUnauthorized() { @Test(expected = GroupAuthorizationException.class) public void testGroupReadUnauthorized() { - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -215,8 +222,8 @@ public void testIllegalGeneration() { coordinator.ensureCoordinatorReady(); // illegal_generation will cause re-partition - subscriptions.subscribe(singleton(topicName), rebalanceListener); - subscriptions.assignFromSubscribed(Collections.singletonList(tp)); + subscriptions.subscribe(singleton(topic1), rebalanceListener); + subscriptions.assignFromSubscribed(Collections.singletonList(t1p)); time.sleep(sessionTimeoutMs); RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat @@ -239,8 +246,8 @@ public void testUnknownConsumerId() { coordinator.ensureCoordinatorReady(); // illegal_generation will cause re-partition - subscriptions.subscribe(singleton(topicName), rebalanceListener); - subscriptions.assignFromSubscribed(Collections.singletonList(tp)); + subscriptions.subscribe(singleton(topic1), rebalanceListener); + subscriptions.assignFromSubscribed(Collections.singletonList(t1p)); time.sleep(sessionTimeoutMs); RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat @@ -282,10 +289,10 @@ public void testCoordinatorDisconnect() { public void testJoinGroupInvalidGroupId() { final String consumerId = "leader"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); // ensure metadata is up-to-date for leader - metadata.setTopics(singletonList(topicName)); + metadata.setTopics(singletonList(topic1)); metadata.update(cluster, time.milliseconds()); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); @@ -300,18 +307,18 @@ public void testJoinGroupInvalidGroupId() { public void testNormalJoinGroupLeader() { final String consumerId = "leader"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); // ensure metadata is up-to-date for leader - metadata.setTopics(singletonList(topicName)); + metadata.setTopics(singletonList(topic1)); metadata.update(cluster, time.milliseconds()); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // normal join group - Map> memberSubscriptions = Collections.singletonMap(consumerId, singletonList(topicName)); - partitionAssignor.prepare(Collections.singletonMap(consumerId, singletonList(tp))); + Map> memberSubscriptions = Collections.singletonMap(consumerId, singletonList(topic1)); + partitionAssignor.prepare(Collections.singletonMap(consumerId, singletonList(t1p))); client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code())); client.prepareResponse(new MockClient.RequestMatcher() { @@ -322,37 +329,79 @@ public boolean matches(AbstractRequest body) { sync.generationId() == 1 && sync.groupAssignment().containsKey(consumerId); } - }, syncGroupResponse(singletonList(tp), Errors.NONE.code())); + }, syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.poll(time.milliseconds()); assertFalse(coordinator.needRejoin()); - assertEquals(singleton(tp), subscriptions.assignedPartitions()); + assertEquals(singleton(t1p), subscriptions.assignedPartitions()); + assertEquals(singleton(topic1), subscriptions.groupSubscription()); assertEquals(1, rebalanceListener.revokedCount); assertEquals(Collections.emptySet(), rebalanceListener.revoked); assertEquals(1, rebalanceListener.assignedCount); - assertEquals(singleton(tp), rebalanceListener.assigned); + assertEquals(singleton(t1p), rebalanceListener.assigned); + } + + @Test + public void testPatternJoinGroupLeader() { + final String consumerId = "leader"; + + subscriptions.subscribe(Pattern.compile("test.*"), rebalanceListener); + + // partially update the metadata with one topic first, + // let the leader to refresh metadata during assignment + metadata.setTopics(singletonList(topic1)); + metadata.update(TestUtils.singletonCluster(topic1, 1), time.milliseconds()); + + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorReady(); + + // normal join group + Map> memberSubscriptions = Collections.singletonMap(consumerId, singletonList(topic1)); + partitionAssignor.prepare(Collections.singletonMap(consumerId, Arrays.asList(t1p, t2p))); + + client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code())); + client.prepareResponse(new MockClient.RequestMatcher() { + @Override + public boolean matches(AbstractRequest body) { + SyncGroupRequest sync = (SyncGroupRequest) body; + return sync.memberId().equals(consumerId) && + sync.generationId() == 1 && + sync.groupAssignment().containsKey(consumerId); + } + }, syncGroupResponse(Arrays.asList(t1p, t2p), Errors.NONE.code())); + // expect client to force updating the metadata, if yes gives it both topics + client.prepareMetadataUpdate(cluster); + + coordinator.poll(time.milliseconds()); + + assertFalse(coordinator.needRejoin()); + assertEquals(2, subscriptions.assignedPartitions().size()); + assertEquals(2, subscriptions.groupSubscription().size()); + assertEquals(2, subscriptions.subscription().size()); + assertEquals(1, rebalanceListener.revokedCount); + assertEquals(Collections.emptySet(), rebalanceListener.revoked); + assertEquals(1, rebalanceListener.assignedCount); + assertEquals(2, rebalanceListener.assigned.size()); } @Test public void testMetadataRefreshDuringRebalance() { final String consumerId = "leader"; - final String otherTopicName = "otherTopic"; - TopicPartition otherPartition = new TopicPartition(otherTopicName, 0); subscriptions.subscribe(Pattern.compile(".*"), rebalanceListener); metadata.needMetadataForAllTopics(true); - metadata.update(cluster, time.milliseconds()); + metadata.update(TestUtils.singletonCluster(topic1, 1), time.milliseconds()); - assertEquals(singleton(topicName), subscriptions.subscription()); + assertEquals(singleton(topic1), subscriptions.subscription()); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - Map> initialSubscription = singletonMap(consumerId, singletonList(topicName)); - partitionAssignor.prepare(singletonMap(consumerId, singletonList(tp))); + Map> initialSubscription = singletonMap(consumerId, singletonList(topic1)); + partitionAssignor.prepare(singletonMap(consumerId, singletonList(t1p))); // the metadata will be updated in flight with a new topic added - final List updatedSubscription = Arrays.asList(topicName, otherTopicName); + final List updatedSubscription = Arrays.asList(topic1, topic2); final Set updatedSubscriptionSet = new HashSet<>(updatedSubscription); client.prepareResponse(joinGroupLeaderResponse(1, consumerId, initialSubscription, Errors.NONE.code())); @@ -365,12 +414,12 @@ public boolean matches(AbstractRequest body) { metadata.update(TestUtils.clusterWith(1, updatedPartitions), time.milliseconds()); return true; } - }, syncGroupResponse(singletonList(tp), Errors.NONE.code())); + }, syncGroupResponse(singletonList(t1p), Errors.NONE.code())); - List newAssignment = Arrays.asList(tp, otherPartition); + List newAssignment = Arrays.asList(t1p, t2p); Set newAssignmentSet = new HashSet<>(newAssignment); - Map> updatedSubscriptions = singletonMap(consumerId, Arrays.asList(topicName, otherTopicName)); + Map> updatedSubscriptions = singletonMap(consumerId, Arrays.asList(topic1, topic2)); partitionAssignor.prepare(singletonMap(consumerId, newAssignment)); // we expect to see a second rebalance with the new-found topics @@ -392,7 +441,7 @@ public boolean matches(AbstractRequest body) { assertEquals(updatedSubscriptionSet, subscriptions.subscription()); assertEquals(newAssignmentSet, subscriptions.assignedPartitions()); assertEquals(2, rebalanceListener.revokedCount); - assertEquals(singleton(tp), rebalanceListener.revoked); + assertEquals(singleton(t1p), rebalanceListener.revoked); assertEquals(2, rebalanceListener.assignedCount); assertEquals(newAssignmentSet, rebalanceListener.assigned); } @@ -401,17 +450,17 @@ public boolean matches(AbstractRequest body) { public void testWakeupDuringJoin() { final String consumerId = "leader"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); // ensure metadata is up-to-date for leader - metadata.setTopics(singletonList(topicName)); + metadata.setTopics(singletonList(topic1)); metadata.update(cluster, time.milliseconds()); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - Map> memberSubscriptions = Collections.singletonMap(consumerId, singletonList(topicName)); - partitionAssignor.prepare(Collections.singletonMap(consumerId, singletonList(tp))); + Map> memberSubscriptions = Collections.singletonMap(consumerId, singletonList(topic1)); + partitionAssignor.prepare(Collections.singletonMap(consumerId, singletonList(t1p))); // prepare only the first half of the join and then trigger the wakeup client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code())); @@ -424,22 +473,22 @@ public void testWakeupDuringJoin() { } // now complete the second half - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.poll(time.milliseconds()); assertFalse(coordinator.needRejoin()); - assertEquals(singleton(tp), subscriptions.assignedPartitions()); + assertEquals(singleton(t1p), subscriptions.assignedPartitions()); assertEquals(1, rebalanceListener.revokedCount); assertEquals(Collections.emptySet(), rebalanceListener.revoked); assertEquals(1, rebalanceListener.assignedCount); - assertEquals(singleton(tp), rebalanceListener.assigned); + assertEquals(singleton(t1p), rebalanceListener.assigned); } @Test public void testNormalJoinGroupFollower() { final String consumerId = "consumer"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -454,28 +503,68 @@ public boolean matches(AbstractRequest body) { sync.generationId() == 1 && sync.groupAssignment().isEmpty(); } - }, syncGroupResponse(singletonList(tp), Errors.NONE.code())); + }, syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); assertFalse(coordinator.needRejoin()); - assertEquals(singleton(tp), subscriptions.assignedPartitions()); + assertEquals(singleton(t1p), subscriptions.assignedPartitions()); + assertEquals(singleton(topic1), subscriptions.groupSubscription()); assertEquals(1, rebalanceListener.revokedCount); + assertEquals(Collections.emptySet(), rebalanceListener.revoked); assertEquals(1, rebalanceListener.assignedCount); - assertEquals(singleton(tp), rebalanceListener.assigned); + assertEquals(singleton(t1p), rebalanceListener.assigned); + } + + @Test + public void testPatternJoinGroupFollower() { + final String consumerId = "consumer"; + + subscriptions.subscribe(Pattern.compile("test.*"), rebalanceListener); + + // partially update the metadata with one topic first, + // let the leader to refresh metadata during assignment + metadata.setTopics(singletonList(topic1)); + metadata.update(TestUtils.singletonCluster(topic1, 1), time.milliseconds()); + + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorReady(); + + // normal join group + client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); + client.prepareResponse(new MockClient.RequestMatcher() { + @Override + public boolean matches(AbstractRequest body) { + SyncGroupRequest sync = (SyncGroupRequest) body; + return sync.memberId().equals(consumerId) && + sync.generationId() == 1 && + sync.groupAssignment().isEmpty(); + } + }, syncGroupResponse(Arrays.asList(t1p, t2p), Errors.NONE.code())); + // expect client to force updating the metadata, if yes gives it both topics + client.prepareMetadataUpdate(cluster); + + coordinator.joinGroupIfNeeded(); + + assertFalse(coordinator.needRejoin()); + assertEquals(2, subscriptions.assignedPartitions().size()); + assertEquals(2, subscriptions.subscription().size()); + assertEquals(1, rebalanceListener.revokedCount); + assertEquals(1, rebalanceListener.assignedCount); + assertEquals(2, rebalanceListener.assigned.size()); } @Test public void testLeaveGroupOnClose() { final String consumerId = "consumer"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); final AtomicBoolean received = new AtomicBoolean(false); @@ -496,13 +585,13 @@ public boolean matches(AbstractRequest body) { public void testMaybeLeaveGroup() { final String consumerId = "consumer"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); final AtomicBoolean received = new AtomicBoolean(false); @@ -526,7 +615,7 @@ public boolean matches(AbstractRequest body) { public void testUnexpectedErrorOnSyncGroup() { final String consumerId = "consumer"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -541,7 +630,7 @@ public void testUnexpectedErrorOnSyncGroup() { public void testUnknownMemberIdOnSyncGroup() { final String consumerId = "consumer"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -558,19 +647,19 @@ public boolean matches(AbstractRequest body) { return joinRequest.memberId().equals(JoinGroupRequest.UNKNOWN_MEMBER_ID); } }, joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); assertFalse(coordinator.needRejoin()); - assertEquals(singleton(tp), subscriptions.assignedPartitions()); + assertEquals(singleton(t1p), subscriptions.assignedPartitions()); } @Test public void testRebalanceInProgressOnSyncGroup() { final String consumerId = "consumer"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -581,19 +670,19 @@ public void testRebalanceInProgressOnSyncGroup() { // then let the full join/sync finish successfully client.prepareResponse(joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); assertFalse(coordinator.needRejoin()); - assertEquals(singleton(tp), subscriptions.assignedPartitions()); + assertEquals(singleton(t1p), subscriptions.assignedPartitions()); } @Test public void testIllegalGenerationOnSyncGroup() { final String consumerId = "consumer"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -610,12 +699,12 @@ public boolean matches(AbstractRequest body) { return joinRequest.memberId().equals(JoinGroupRequest.UNKNOWN_MEMBER_ID); } }, joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); assertFalse(coordinator.needRejoin()); - assertEquals(singleton(tp), subscriptions.assignedPartitions()); + assertEquals(singleton(t1p), subscriptions.assignedPartitions()); } @Test @@ -623,27 +712,27 @@ public void testMetadataChangeTriggersRebalance() { final String consumerId = "consumer"; // ensure metadata is up-to-date for leader - metadata.setTopics(singletonList(topicName)); + metadata.setTopics(singletonList(topic1)); metadata.update(cluster, time.milliseconds()); - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - Map> memberSubscriptions = Collections.singletonMap(consumerId, singletonList(topicName)); - partitionAssignor.prepare(Collections.singletonMap(consumerId, singletonList(tp))); + Map> memberSubscriptions = Collections.singletonMap(consumerId, singletonList(topic1)); + partitionAssignor.prepare(Collections.singletonMap(consumerId, singletonList(t1p))); // the leader is responsible for picking up metadata changes and forcing a group rebalance client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.poll(time.milliseconds()); assertFalse(coordinator.needRejoin()); // a new partition is added to the topic - metadata.update(TestUtils.singletonCluster(topicName, 2), time.milliseconds()); + metadata.update(TestUtils.singletonCluster(topic1, 2), time.milliseconds()); // we should detect the change and ask for reassignment assertTrue(coordinator.needRejoin()); @@ -670,7 +759,7 @@ public void testUpdateMetadataDuringRebalance() { // prepare initial rebalance Map> memberSubscriptions = Collections.singletonMap(consumerId, topics); - partitionAssignor.prepare(Collections.singletonMap(consumerId, Arrays.asList(tp1))); + partitionAssignor.prepare(Collections.singletonMap(consumerId, Collections.singletonList(tp1))); client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code())); client.prepareResponse(new MockClient.RequestMatcher() { @@ -689,7 +778,7 @@ public boolean matches(AbstractRequest body) { } return false; } - }, syncGroupResponse(Arrays.asList(tp1), Errors.NONE.code())); + }, syncGroupResponse(Collections.singletonList(tp1), Errors.NONE.code())); // the metadata update should trigger a second rebalance client.prepareResponse(joinGroupLeaderResponse(2, consumerId, memberSubscriptions, Errors.NONE.code())); @@ -725,36 +814,36 @@ public void testIncludeInternalTopicsConfigOption() { public void testRejoinGroup() { String otherTopic = "otherTopic"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // join the group once client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); assertEquals(1, rebalanceListener.revokedCount); assertTrue(rebalanceListener.revoked.isEmpty()); assertEquals(1, rebalanceListener.assignedCount); - assertEquals(singleton(tp), rebalanceListener.assigned); + assertEquals(singleton(t1p), rebalanceListener.assigned); // and join the group again - subscriptions.subscribe(new HashSet<>(Arrays.asList(topicName, otherTopic)), rebalanceListener); + subscriptions.subscribe(new HashSet<>(Arrays.asList(topic1, otherTopic)), rebalanceListener); client.prepareResponse(joinGroupFollowerResponse(2, "consumer", "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); assertEquals(2, rebalanceListener.revokedCount); - assertEquals(singleton(tp), rebalanceListener.revoked); + assertEquals(singleton(t1p), rebalanceListener.revoked); assertEquals(2, rebalanceListener.assignedCount); - assertEquals(singleton(tp), rebalanceListener.assigned); + assertEquals(singleton(t1p), rebalanceListener.assigned); } @Test public void testDisconnectInJoin() { - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -763,19 +852,19 @@ public void testDisconnectInJoin() { client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code()), true); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); assertFalse(coordinator.needRejoin()); - assertEquals(singleton(tp), subscriptions.assignedPartitions()); + assertEquals(singleton(t1p), subscriptions.assignedPartitions()); assertEquals(1, rebalanceListener.revokedCount); assertEquals(1, rebalanceListener.assignedCount); - assertEquals(singleton(tp), rebalanceListener.assigned); + assertEquals(singleton(t1p), rebalanceListener.assigned); } @Test(expected = ApiException.class) public void testInvalidSessionTimeout() { - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -787,19 +876,19 @@ public void testInvalidSessionTimeout() { @Test public void testCommitOffsetOnly() { - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); AtomicBoolean success = new AtomicBoolean(false); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), callback(success)); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), callback(success)); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(success.get()); - assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals(100L, subscriptions.committed(t1p).offset()); } @Test @@ -809,22 +898,22 @@ public void testAutoCommitDynamicAssignment() { ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, ConsumerConfig.DEFAULT_EXCLUDE_INTERNAL_TOPICS, true); - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); - subscriptions.seek(tp, 100); + subscriptions.seek(t1p, 100); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); time.sleep(autoCommitIntervalMs); coordinator.poll(time.milliseconds()); - assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals(100L, subscriptions.committed(t1p).offset()); } @Test @@ -834,7 +923,7 @@ public void testAutoCommitDynamicAssignmentRebalance() { ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, ConsumerConfig.DEFAULT_EXCLUDE_INTERNAL_TOPICS, true); - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -844,16 +933,16 @@ public void testAutoCommitDynamicAssignmentRebalance() { consumerClient.poll(0); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); - subscriptions.seek(tp, 100); + subscriptions.seek(t1p, 100); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); time.sleep(autoCommitIntervalMs); coordinator.poll(time.milliseconds()); - assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals(100L, subscriptions.committed(t1p).offset()); } @Test @@ -861,17 +950,17 @@ public void testAutoCommitManualAssignment() { ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, ConsumerConfig.DEFAULT_EXCLUDE_INTERNAL_TOPICS, true); - subscriptions.assignFromUser(singleton(tp)); - subscriptions.seek(tp, 100); + subscriptions.assignFromUser(singleton(t1p)); + subscriptions.seek(t1p, 100); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); time.sleep(autoCommitIntervalMs); coordinator.poll(time.milliseconds()); - assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals(100L, subscriptions.committed(t1p).offset()); } @Test @@ -879,15 +968,15 @@ public void testAutoCommitManualAssignmentCoordinatorUnknown() { ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, ConsumerConfig.DEFAULT_EXCLUDE_INTERNAL_TOPICS, true); - subscriptions.assignFromUser(singleton(tp)); - subscriptions.seek(tp, 100); + subscriptions.assignFromUser(singleton(t1p)); + subscriptions.seek(t1p, 100); // no commit initially since coordinator is unknown consumerClient.poll(0); time.sleep(autoCommitIntervalMs); consumerClient.poll(0); - assertNull(subscriptions.committed(tp)); + assertNull(subscriptions.committed(t1p)); // now find the coordinator client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); @@ -895,28 +984,28 @@ public void testAutoCommitManualAssignmentCoordinatorUnknown() { // sleep only for the retry backoff time.sleep(retryBackoffMs); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); coordinator.poll(time.milliseconds()); - assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals(100L, subscriptions.committed(t1p).offset()); } @Test public void testCommitOffsetMetadata() { - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); AtomicBoolean success = new AtomicBoolean(false); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "hello")), callback(success)); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L, "hello")), callback(success)); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(success.get()); - assertEquals(100L, subscriptions.committed(tp).offset()); - assertEquals("hello", subscriptions.committed(tp).metadata()); + assertEquals(100L, subscriptions.committed(t1p).offset()); + assertEquals("hello", subscriptions.committed(t1p).metadata()); } @Test @@ -924,8 +1013,8 @@ public void testCommitOffsetAsyncWithDefaultCallback() { int invokedBeforeTest = defaultOffsetCommitCallback.invoked; client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), null); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), null); coordinator.invokeCompletedOffsetCommitCallbacks(); assertEquals(invokedBeforeTest + 1, defaultOffsetCommitCallback.invoked); assertNull(defaultOffsetCommitCallback.exception); @@ -934,20 +1023,23 @@ public void testCommitOffsetAsyncWithDefaultCallback() { @Test public void testCommitAfterLeaveGroup() { // enable auto-assignment - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); + + client.prepareMetadataUpdate(cluster); + coordinator.joinGroupIfNeeded(); // now switch to manual assignment client.prepareResponse(new LeaveGroupResponse(Errors.NONE.code())); subscriptions.unsubscribe(); coordinator.maybeLeaveGroup(); - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); // the client should not reuse generation/memberId from auto-subscribed generation client.prepareResponse(new MockClient.RequestMatcher() { @@ -957,10 +1049,10 @@ public boolean matches(AbstractRequest body) { return commitRequest.memberId().equals(OffsetCommitRequest.DEFAULT_MEMBER_ID) && commitRequest.generationId() == OffsetCommitRequest.DEFAULT_GENERATION_ID; } - }, offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + }, offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); AtomicBoolean success = new AtomicBoolean(false); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), callback(success)); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), callback(success)); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(success.get()); } @@ -970,8 +1062,8 @@ public void testCommitOffsetAsyncFailedWithDefaultCallback() { int invokedBeforeTest = defaultOffsetCommitCallback.invoked; client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), null); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), null); coordinator.invokeCompletedOffsetCommitCallbacks(); assertEquals(invokedBeforeTest + 1, defaultOffsetCommitCallback.invoked); assertTrue(defaultOffsetCommitCallback.exception instanceof RetriableCommitFailedException); @@ -984,8 +1076,8 @@ public void testCommitOffsetAsyncCoordinatorNotAvailable() { // async commit with coordinator not available MockCommitCallback cb = new MockCommitCallback(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), cb); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(coordinator.coordinatorUnknown()); @@ -1000,8 +1092,8 @@ public void testCommitOffsetAsyncNotCoordinator() { // async commit with not coordinator MockCommitCallback cb = new MockCommitCallback(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_GROUP.code()))); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NOT_COORDINATOR_FOR_GROUP.code()))); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), cb); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(coordinator.coordinatorUnknown()); @@ -1016,8 +1108,8 @@ public void testCommitOffsetAsyncDisconnected() { // async commit with coordinator disconnected MockCommitCallback cb = new MockCommitCallback(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code())), true); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), cb); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(coordinator.coordinatorUnknown()); @@ -1031,10 +1123,10 @@ public void testCommitOffsetSyncNotCoordinator() { coordinator.ensureCoordinatorReady(); // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_GROUP.code()))); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NOT_COORDINATOR_FOR_GROUP.code()))); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), Long.MAX_VALUE); } @Test @@ -1043,10 +1135,10 @@ public void testCommitOffsetSyncCoordinatorNotAvailable() { coordinator.ensureCoordinatorReady(); // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), Long.MAX_VALUE); } @Test @@ -1055,10 +1147,10 @@ public void testCommitOffsetSyncCoordinatorDisconnected() { coordinator.ensureCoordinatorReady(); // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code())), true); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), Long.MAX_VALUE); } @Test(expected = KafkaException.class) @@ -1066,8 +1158,8 @@ public void testCommitUnknownTopicOrPartition() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.UNKNOWN_TOPIC_OR_PARTITION.code()))); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.UNKNOWN_TOPIC_OR_PARTITION.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); } @Test(expected = OffsetMetadataTooLarge.class) @@ -1076,8 +1168,8 @@ public void testCommitOffsetMetadataTooLarge() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.OFFSET_METADATA_TOO_LARGE.code()))); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.OFFSET_METADATA_TOO_LARGE.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); } @Test(expected = CommitFailedException.class) @@ -1086,8 +1178,8 @@ public void testCommitOffsetIllegalGeneration() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.ILLEGAL_GENERATION.code()))); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.ILLEGAL_GENERATION.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); } @Test(expected = CommitFailedException.class) @@ -1096,8 +1188,8 @@ public void testCommitOffsetUnknownMemberId() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.UNKNOWN_MEMBER_ID.code()))); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.UNKNOWN_MEMBER_ID.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); } @Test(expected = CommitFailedException.class) @@ -1106,8 +1198,8 @@ public void testCommitOffsetRebalanceInProgress() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.REBALANCE_IN_PROGRESS.code()))); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.REBALANCE_IN_PROGRESS.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); } @Test(expected = KafkaException.class) @@ -1116,21 +1208,21 @@ public void testCommitOffsetSyncCallbackWithNonRetriableException() { coordinator.ensureCoordinatorReady(); // sync commit with invalid partitions should throw if we have no callback - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.UNKNOWN.code())), false); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.UNKNOWN.code())), false); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), Long.MAX_VALUE); } @Test(expected = IllegalArgumentException.class) public void testCommitSyncNegativeOffset() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(-1L)), Long.MAX_VALUE); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(-1L)), Long.MAX_VALUE); } @Test public void testCommitAsyncNegativeOffset() { int invokedBeforeTest = defaultOffsetCommitCallback.invoked; client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(-1L)), null); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(-1L)), null); coordinator.invokeCompletedOffsetCommitCallbacks(); assertEquals(invokedBeforeTest + 1, defaultOffsetCommitCallback.invoked); assertTrue(defaultOffsetCommitCallback.exception instanceof IllegalArgumentException); @@ -1141,12 +1233,12 @@ public void testRefreshOffset() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); subscriptions.needRefreshCommits(); - client.prepareResponse(offsetFetchResponse(tp, Errors.NONE, "", 100L)); + client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L)); coordinator.refreshCommittedOffsetsIfNeeded(); assertFalse(subscriptions.refreshCommitsNeeded()); - assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals(100L, subscriptions.committed(t1p).offset()); } @Test @@ -1154,13 +1246,13 @@ public void testRefreshOffsetLoadInProgress() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(Errors.GROUP_LOAD_IN_PROGRESS)); - client.prepareResponse(offsetFetchResponse(tp, Errors.NONE, "", 100L)); + client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L)); coordinator.refreshCommittedOffsetsIfNeeded(); assertFalse(subscriptions.refreshCommitsNeeded()); - assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals(100L, subscriptions.committed(t1p).offset()); } @Test @@ -1168,7 +1260,7 @@ public void testRefreshOffsetsGroupNotAuthorized() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(Errors.GROUP_AUTHORIZATION_FAILED)); try { @@ -1184,9 +1276,9 @@ public void testRefreshOffsetUnknownTopicOrPartition() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); subscriptions.needRefreshCommits(); - client.prepareResponse(offsetFetchResponse(tp, Errors.UNKNOWN_TOPIC_OR_PARTITION, "", 100L)); + client.prepareResponse(offsetFetchResponse(t1p, Errors.UNKNOWN_TOPIC_OR_PARTITION, "", 100L)); coordinator.refreshCommittedOffsetsIfNeeded(); } @@ -1195,14 +1287,14 @@ public void testRefreshOffsetNotCoordinatorForConsumer() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR_FOR_GROUP)); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); - client.prepareResponse(offsetFetchResponse(tp, Errors.NONE, "", 100L)); + client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L)); coordinator.refreshCommittedOffsetsIfNeeded(); assertFalse(subscriptions.refreshCommitsNeeded()); - assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals(100L, subscriptions.committed(t1p).offset()); } @Test @@ -1210,12 +1302,12 @@ public void testRefreshOffsetWithNoFetchableOffsets() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); subscriptions.needRefreshCommits(); - client.prepareResponse(offsetFetchResponse(tp, Errors.NONE, "", -1L)); + client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", -1L)); coordinator.refreshCommittedOffsetsIfNeeded(); assertFalse(subscriptions.refreshCommitsNeeded()); - assertEquals(null, subscriptions.committed(tp)); + assertEquals(null, subscriptions.committed(t1p)); } @Test @@ -1341,14 +1433,14 @@ private ConsumerCoordinator prepareCoordinatorForCloseTest(boolean useGroupManag client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); if (useGroupManagement) { - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); } else - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); - subscriptions.seek(tp, 100); + subscriptions.seek(t1p, 100); coordinator.poll(time.milliseconds()); return coordinator; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java index ef7575444fece..55bf2a357db28 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java @@ -29,7 +29,6 @@ import java.util.HashSet; import java.util.regex.Pattern; -import static java.util.Arrays.asList; import static java.util.Collections.singleton; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -77,7 +76,7 @@ public void partitionAssignmentChangeOnTopicSubscription() { // assigned partitions should remain unchanged assertTrue(state.assignedPartitions().isEmpty()); - state.assignFromSubscribed(asList(t1p0)); + state.assignFromSubscribed(Collections.singletonList(t1p0)); // assigned partitions should immediately change assertEquals(singleton(t1p0), state.assignedPartitions()); @@ -96,21 +95,32 @@ public void partitionAssignmentChangeOnPatternSubscription() { // assigned partitions should remain unchanged assertTrue(state.assignedPartitions().isEmpty()); - state.subscribeFromPattern(new HashSet<>(Arrays.asList(topic, topic1))); + state.subscribeFromPattern(new HashSet<>(Collections.singletonList(topic))); // assigned partitions should remain unchanged assertTrue(state.assignedPartitions().isEmpty()); - state.assignFromSubscribed(asList(tp1)); + state.assignFromSubscribed(Collections.singletonList(tp1)); // assigned partitions should immediately change assertEquals(singleton(tp1), state.assignedPartitions()); + assertEquals(singleton(topic), state.subscription()); + + state.assignFromSubscribed(Collections.singletonList(t1p0)); + // assigned partitions should immediately change + assertEquals(singleton(t1p0), state.assignedPartitions()); + assertEquals(singleton(topic), state.subscription()); state.subscribe(Pattern.compile(".*t"), rebalanceListener); // assigned partitions should remain unchanged - assertEquals(singleton(tp1), state.assignedPartitions()); + assertEquals(singleton(t1p0), state.assignedPartitions()); state.subscribeFromPattern(singleton(topic)); // assigned partitions should remain unchanged - assertEquals(singleton(tp1), state.assignedPartitions()); + assertEquals(singleton(t1p0), state.assignedPartitions()); + + state.assignFromSubscribed(Collections.singletonList(tp0)); + // assigned partitions should immediately change + assertEquals(singleton(tp0), state.assignedPartitions()); + assertEquals(singleton(topic), state.subscription()); state.unsubscribe(); // assigned partitions should immediately change @@ -139,11 +149,11 @@ public void topicSubscription() { assertEquals(1, state.subscription().size()); assertTrue(state.assignedPartitions().isEmpty()); assertTrue(state.partitionsAutoAssigned()); - state.assignFromSubscribed(asList(tp0)); + state.assignFromSubscribed(Collections.singletonList(tp0)); state.seek(tp0, 1); state.committed(tp0, new OffsetAndMetadata(1)); assertAllPositions(tp0, 1L); - state.assignFromSubscribed(asList(tp1)); + state.assignFromSubscribed(Collections.singletonList(tp1)); assertTrue(state.isAssigned(tp1)); assertFalse(state.isAssigned(tp0)); assertFalse(state.isFetchable(tp1)); @@ -173,20 +183,28 @@ public void commitOffsetMetadata() { @Test(expected = IllegalStateException.class) public void invalidPositionUpdate() { state.subscribe(singleton(topic), rebalanceListener); - state.assignFromSubscribed(asList(tp0)); + state.assignFromSubscribed(Collections.singletonList(tp0)); state.position(tp0, 0); } + @Test(expected = IllegalArgumentException.class) + public void cantAssignPartitionForUnsubscribedTopics() { + state.subscribe(singleton(topic), rebalanceListener); + state.assignFromSubscribed(Collections.singletonList(t1p0)); + } + + @Test(expected = IllegalArgumentException.class) + public void cantAssignPartitionForUnmatchedPattern() { + state.subscribe(Pattern.compile(".*t"), rebalanceListener); + state.subscribeFromPattern(new HashSet<>(Collections.singletonList(topic))); + state.assignFromSubscribed(Collections.singletonList(t1p0)); + } + @Test(expected = IllegalStateException.class) public void cantChangePositionForNonAssignedPartition() { state.position(tp0, 1); } - public void assertAllPositions(TopicPartition tp, Long offset) { - assertEquals(offset.longValue(), state.committed(tp).offset()); - assertEquals(offset, state.position(tp)); - } - @Test(expected = IllegalStateException.class) public void cantSubscribeTopicAndPattern() { state.subscribe(singleton(topic), rebalanceListener); @@ -240,7 +258,7 @@ public void unsubscribeUserSubscribe() { public void unsubscription() { state.subscribe(Pattern.compile(".*"), rebalanceListener); state.subscribeFromPattern(new HashSet<>(Arrays.asList(topic, topic1))); - state.assignFromSubscribed(asList(tp1)); + state.assignFromSubscribed(Collections.singletonList(tp1)); assertEquals(singleton(tp1), state.assignedPartitions()); state.unsubscribe(); @@ -255,6 +273,11 @@ public void unsubscription() { assertTrue(state.assignedPartitions().isEmpty()); } + private void assertAllPositions(TopicPartition tp, Long offset) { + assertEquals(offset.longValue(), state.committed(tp).offset()); + assertEquals(offset, state.position(tp)); + } + private static class MockRebalanceListener implements ConsumerRebalanceListener { public Collection revoked; public Collection assigned; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java index 31a3dc63b3416..bcffce2dcb227 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java @@ -37,12 +37,6 @@ public abstract class AbstractStream { protected final String name; protected final Set sourceNodes; - public AbstractStream(AbstractStream stream) { - this.topology = stream.topology; - this.name = stream.name; - this.sourceNodes = stream.sourceNodes; - } - public AbstractStream(KStreamBuilder topology, String name, Set sourceNodes) { if (sourceNodes == null || sourceNodes.isEmpty()) { throw new IllegalArgumentException("parameter must not be null or empty"); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java index b25fcad3d1f2f..81f4302429bff 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java @@ -49,6 +49,7 @@ import java.util.Set; import java.util.regex.Pattern; + /** * A component that is used to build a {@link ProcessorTopology}. A topology contains an acyclic graph of sources, processors, * and sinks. A {@link SourceNode source} is a node in the graph that consumes one or more Kafka topics and forwards them to @@ -81,7 +82,7 @@ public class TopologyBuilder { private final List> copartitionSourceGroups = new ArrayList<>(); // map from source processor names to subscribed topics (without application-id prefix for internal topics) - private final HashMap nodeToSourceTopics = new HashMap<>(); + private final HashMap> nodeToSourceTopics = new HashMap<>(); // map from source processor names to regex subscription patterns private final HashMap nodeToSourcePatterns = new LinkedHashMap<>(); @@ -146,11 +147,11 @@ private static abstract class NodeFactory { } private static class ProcessorNodeFactory extends NodeFactory { - public final String[] parents; - private final ProcessorSupplier supplier; + private final String[] parents; + private final ProcessorSupplier supplier; private final Set stateStoreNames = new HashSet<>(); - public ProcessorNodeFactory(String name, String[] parents, ProcessorSupplier supplier) { + ProcessorNodeFactory(String name, String[] parents, ProcessorSupplier supplier) { super(name); this.parents = parents.clone(); this.supplier = supplier; @@ -160,37 +161,32 @@ public void addStateStore(String stateStoreName) { stateStoreNames.add(stateStoreName); } - @SuppressWarnings("unchecked") @Override public ProcessorNode build() { - return new ProcessorNode(name, supplier.get(), stateStoreNames); + return new ProcessorNode<>(name, supplier.get(), stateStoreNames); } } private class SourceNodeFactory extends NodeFactory { - private final String[] topics; - public final Pattern pattern; - private Deserializer keyDeserializer; - private Deserializer valDeserializer; + private final List topics; + private final Pattern pattern; + private final Deserializer keyDeserializer; + private final Deserializer valDeserializer; - private SourceNodeFactory(String name, String[] topics, Pattern pattern, Deserializer keyDeserializer, Deserializer valDeserializer) { + private SourceNodeFactory(String name, String[] topics, Pattern pattern, Deserializer keyDeserializer, Deserializer valDeserializer) { super(name); - this.topics = topics != null ? topics.clone() : null; + this.topics = topics != null ? Arrays.asList(topics) : null; this.pattern = pattern; this.keyDeserializer = keyDeserializer; this.valDeserializer = valDeserializer; } - String[] getTopics() { - return topics; - } - - String[] getTopics(Collection subscribedTopics) { + List getTopics(Collection subscribedTopics) { // if it is subscribed via patterns, it is possible that the topic metadata has not been updated // yet and hence the map from source node to topics is stale, in this case we put the pattern as a place holder; // this should only happen for debugging since during runtime this function should always be called after the metadata has updated. if (subscribedTopics.isEmpty()) - return new String[] {"Pattern[" + pattern + "]"}; + return Collections.singletonList("Pattern[" + pattern + "]"); List matchedTopics = new ArrayList<>(); for (String update : subscribedTopics) { @@ -207,21 +203,20 @@ String[] getTopics(Collection subscribedTopics) { matchedTopics.add(update); } } - return matchedTopics.toArray(new String[matchedTopics.size()]); + return matchedTopics; } - @SuppressWarnings("unchecked") @Override public ProcessorNode build() { - final String[] sourceTopics = nodeToSourceTopics.get(name); + final List sourceTopics = nodeToSourceTopics.get(name); // if it is subscribed via patterns, it is possible that the topic metadata has not been updated // yet and hence the map from source node to topics is stale, in this case we put the pattern as a place holder; // this should only happen for debugging since during runtime this function should always be called after the metadata has updated. if (sourceTopics == null) - return new SourceNode(name, new String[] {"Pattern[" + pattern + "]"}, keyDeserializer, valDeserializer); + return new SourceNode<>(name, Collections.singletonList("Pattern[" + pattern + "]"), keyDeserializer, valDeserializer); else - return new SourceNode(name, maybeDecorateInternalSourceTopics(sourceTopics).toArray(new String[sourceTopics.length]), keyDeserializer, valDeserializer); + return new SourceNode<>(name, maybeDecorateInternalSourceTopics(sourceTopics), keyDeserializer, valDeserializer); } private boolean isMatch(String topic) { @@ -229,14 +224,14 @@ private boolean isMatch(String topic) { } } - private class SinkNodeFactory extends NodeFactory { - public final String[] parents; - public final String topic; - private Serializer keySerializer; - private Serializer valSerializer; - private final StreamPartitioner partitioner; + private class SinkNodeFactory extends NodeFactory { + private final String[] parents; + private final String topic; + private final Serializer keySerializer; + private final Serializer valSerializer; + private final StreamPartitioner partitioner; - private SinkNodeFactory(String name, String[] parents, String topic, Serializer keySerializer, Serializer valSerializer, StreamPartitioner partitioner) { + private SinkNodeFactory(String name, String[] parents, String topic, Serializer keySerializer, Serializer valSerializer, StreamPartitioner partitioner) { super(name); this.parents = parents.clone(); this.topic = topic; @@ -245,14 +240,13 @@ private SinkNodeFactory(String name, String[] parents, String topic, Serializer this.partitioner = partitioner; } - @SuppressWarnings("unchecked") @Override public ProcessorNode build() { if (internalTopicNames.contains(topic)) { // prefix the internal topic name with the application id - return new SinkNode(name, decorateTopic(topic), keySerializer, valSerializer, partitioner); + return new SinkNode<>(name, decorateTopic(topic), keySerializer, valSerializer, partitioner); } else { - return new SinkNode(name, topic, keySerializer, valSerializer, partitioner); + return new SinkNode<>(name, topic, keySerializer, valSerializer, partitioner); } } } @@ -263,7 +257,7 @@ public static class TopicsInfo { public Map stateChangelogTopics; public Map repartitionSourceTopics; - public TopicsInfo(Set sinkTopics, Set sourceTopics, Map repartitionSourceTopics, Map stateChangelogTopics) { + TopicsInfo(Set sinkTopics, Set sourceTopics, Map repartitionSourceTopics, Map stateChangelogTopics) { this.sinkTopics = sinkTopics; this.sourceTopics = sourceTopics; this.stateChangelogTopics = stateChangelogTopics; @@ -312,8 +306,8 @@ public TopologyBuilder() {} /** * Set the applicationId to be used for auto-generated internal topics. * - * This is required before calling {@link #sourceTopics}, {@link #topicGroups}, - * {@link #copartitionSources}, {@link #stateStoreNameToSourceTopics} and {@link #build(Integer)}. + * This is required before calling {@link #topicGroups}, {@link #copartitionSources}, + * {@link #stateStoreNameToSourceTopics} and {@link #build(Integer)}. * * @param applicationId the streams applicationId. Should be the same as set by * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG} @@ -337,7 +331,7 @@ public synchronized final TopologyBuilder setApplicationId(String applicationId) * @return this builder instance so methods can be chained together; never null */ public synchronized final TopologyBuilder addSource(String name, String... topics) { - return addSource(null, name, (Deserializer) null, (Deserializer) null, topics); + return addSource(null, name, null, null, topics); } /** @@ -353,7 +347,7 @@ public synchronized final TopologyBuilder addSource(String name, String... topic * @return this builder instance so methods can be chained together; never null */ public synchronized final TopologyBuilder addSource(AutoOffsetReset offsetReset, String name, String... topics) { - return addSource(offsetReset, name, (Deserializer) null, (Deserializer) null, topics); + return addSource(offsetReset, name, null, null, topics); } @@ -370,7 +364,7 @@ public synchronized final TopologyBuilder addSource(AutoOffsetReset offsetReset, * @return this builder instance so methods can be chained together; never null */ public synchronized final TopologyBuilder addSource(String name, Pattern topicPattern) { - return addSource(null, name, (Deserializer) null, (Deserializer) null, topicPattern); + return addSource(null, name, null, null, topicPattern); } /** @@ -387,7 +381,7 @@ public synchronized final TopologyBuilder addSource(String name, Pattern topicPa * @return this builder instance so methods can be chained together; never null */ public synchronized final TopologyBuilder addSource(AutoOffsetReset offsetReset, String name, Pattern topicPattern) { - return addSource(offsetReset, name, (Deserializer) null, (Deserializer) null, topicPattern); + return addSource(offsetReset, name, null, null, topicPattern); } @@ -445,7 +439,7 @@ public synchronized final TopologyBuilder addSource(AutoOffsetReset offsetReset, } nodeFactories.put(name, new SourceNodeFactory(name, topics, null, keyDeserializer, valDeserializer)); - nodeToSourceTopics.put(name, topics.clone()); + nodeToSourceTopics.put(name, Arrays.asList(topics)); nodeGrouper.add(name); return this; @@ -470,7 +464,7 @@ public synchronized final TopologyBuilder addSource(AutoOffsetReset offsetReset, * @param topic the topic to source the data from * @param processorName the name of the {@link ProcessorSupplier} * @param stateUpdateSupplier the instance of {@link ProcessorSupplier} - * @return + * @return this builder instance so methods can be chained together; never null */ public synchronized TopologyBuilder addGlobalStore(final StateStore store, final String sourceName, @@ -499,7 +493,7 @@ public synchronized TopologyBuilder addGlobalStore(final StateStore store, globalTopics.add(topic); final String[] topics = {topic}; nodeFactories.put(sourceName, new SourceNodeFactory(sourceName, topics, null, keyDeserializer, valueDeserializer)); - nodeToSourceTopics.put(sourceName, topics.clone()); + nodeToSourceTopics.put(sourceName, Arrays.asList(topics)); nodeGrouper.add(sourceName); final String[] parents = {sourceName}; @@ -612,7 +606,7 @@ public synchronized final TopologyBuilder addSource(AutoOffsetReset offsetReset, * @see #addSink(String, String, Serializer, Serializer, StreamPartitioner, String...) */ public synchronized final TopologyBuilder addSink(String name, String topic, String... parentNames) { - return addSink(name, topic, (Serializer) null, (Serializer) null, parentNames); + return addSink(name, topic, null, null, parentNames); } /** @@ -639,7 +633,7 @@ public synchronized final TopologyBuilder addSink(String name, String topic, Str * @see #addSink(String, String, Serializer, Serializer, StreamPartitioner, String...) */ public synchronized final TopologyBuilder addSink(String name, String topic, StreamPartitioner partitioner, String... parentNames) { - return addSink(name, topic, (Serializer) null, (Serializer) null, partitioner, parentNames); + return addSink(name, topic, null, null, partitioner, parentNames); } /** @@ -662,7 +656,7 @@ public synchronized final TopologyBuilder addSink(String name, String topic, Str * @see #addSink(String, String, Serializer, Serializer, StreamPartitioner, String...) */ public synchronized final TopologyBuilder addSink(String name, String topic, Serializer keySerializer, Serializer valSerializer, String... parentNames) { - return addSink(name, topic, keySerializer, valSerializer, (StreamPartitioner) null, parentNames); + return addSink(name, topic, keySerializer, valSerializer, null, parentNames); } /** @@ -703,7 +697,7 @@ public synchronized final TopologyBuilder addSink(String name, String top } } - nodeFactories.put(name, new SinkNodeFactory(name, parentNames, topic, keySerializer, valSerializer, partitioner)); + nodeFactories.put(name, new SinkNodeFactory<>(name, parentNames, topic, keySerializer, valSerializer, partitioner)); nodeToSinkTopic.put(name, topic); nodeGrouper.add(name); nodeGrouper.unite(name, parentNames); @@ -876,7 +870,7 @@ private Set findSourceTopicsForProcessorParents(String [] parents) { for (String parent : parents) { NodeFactory nodeFactory = nodeFactories.get(parent); if (nodeFactory instanceof SourceNodeFactory) { - sourceTopics.addAll(Arrays.asList(((SourceNodeFactory) nodeFactory).getTopics())); + sourceTopics.addAll(((SourceNodeFactory) nodeFactory).topics); } else if (nodeFactory instanceof ProcessorNodeFactory) { sourceTopics.addAll(findSourceTopicsForProcessorParents(((ProcessorNodeFactory) nodeFactory).parents)); } @@ -1013,8 +1007,8 @@ private Set globalNodeGroups() { for (String node : nodes) { final NodeFactory nodeFactory = nodeFactories.get(node); if (nodeFactory instanceof SourceNodeFactory) { - final String[] topics = ((SourceNodeFactory) nodeFactory).getTopics(); - if (topics != null && topics.length == 1 && globalTopics.contains(topics[0])) { + final List topics = ((SourceNodeFactory) nodeFactory).topics; + if (topics != null && topics.size() == 1 && globalTopics.contains(topics.get(0))) { globalGroups.addAll(nodes); } } @@ -1023,7 +1017,6 @@ private Set globalNodeGroups() { return globalGroups; } - @SuppressWarnings("unchecked") private ProcessorTopology build(Set nodeGroup) { List processorNodes = new ArrayList<>(nodeFactories.size()); Map processorMap = new HashMap<>(); @@ -1040,7 +1033,8 @@ private ProcessorTopology build(Set nodeGroup) { if (factory instanceof ProcessorNodeFactory) { for (String parent : ((ProcessorNodeFactory) factory).parents) { - processorMap.get(parent).addChild(node); + ProcessorNode parentNode = processorMap.get(parent); + parentNode.addChild(node); } for (String stateStoreName : ((ProcessorNodeFactory) factory).stateStoreNames) { if (!stateStoreMap.containsKey(stateStoreName)) { @@ -1063,10 +1057,10 @@ private ProcessorTopology build(Set nodeGroup) { } } } else if (factory instanceof SourceNodeFactory) { - SourceNodeFactory sourceNodeFactory = (SourceNodeFactory) factory; - String[] topics = (sourceNodeFactory.pattern != null) ? + final SourceNodeFactory sourceNodeFactory = (SourceNodeFactory) factory; + final List topics = (sourceNodeFactory.pattern != null) ? sourceNodeFactory.getTopics(subscriptionUpdates.getUpdates()) : - sourceNodeFactory.getTopics(); + sourceNodeFactory.topics; for (String topic : topics) { if (internalTopicNames.contains(topic)) { @@ -1077,7 +1071,8 @@ private ProcessorTopology build(Set nodeGroup) { } } } else if (factory instanceof SinkNodeFactory) { - SinkNodeFactory sinkNodeFactory = (SinkNodeFactory) factory; + final SinkNodeFactory sinkNodeFactory = (SinkNodeFactory) factory; + for (String parent : sinkNodeFactory.parents) { processorMap.get(parent).addChild(node); if (internalTopicNames.contains(sinkNodeFactory.topic)) { @@ -1105,13 +1100,6 @@ public Map globalStateStores() { return Collections.unmodifiableMap(globalStateStores); } - private StateStore getStateStore(final String stateStoreName) { - if (stateFactories.containsKey(stateStoreName)) { - return stateFactories.get(stateStoreName).supplier.get(); - } - return globalStateStores.get(stateStoreName); - } - /** * Returns the map of topic groups keyed by the group id. * A topic group is a group of topics in the same task. @@ -1131,7 +1119,7 @@ public synchronized Map topicGroups() { Map stateChangelogTopics = new HashMap<>(); for (String node : entry.getValue()) { // if the node is a source node, add to the source topics - String[] topics = nodeToSourceTopics.get(node); + List topics = nodeToSourceTopics.get(node); if (topics != null) { // if some of the topics are internal, add them to the internal topics for (String topic : topics) { @@ -1196,7 +1184,7 @@ private void setRegexMatchedTopicsToSourceNodes() { } } - private InternalTopicConfig createInternalTopicConfig(final StateStoreSupplier supplier, final String name) { + private InternalTopicConfig createInternalTopicConfig(final StateStoreSupplier supplier, final String name) { if (!(supplier instanceof WindowStoreSupplier)) { return new InternalTopicConfig(name, Collections.singleton(InternalTopicConfig.CleanupPolicy.compact), supplier.logConfig()); } @@ -1210,27 +1198,13 @@ private InternalTopicConfig createInternalTopicConfig(final StateStoreSupplier s return config; } - - /** - * Get the names of topics that are to be consumed by the source nodes created by this builder. - * @return the unmodifiable set of topic names used by source nodes, which changes as new sources are added; never null - */ - public synchronized Set sourceTopics() { - Set topics = maybeDecorateInternalSourceTopics(sourceTopicNames); - return Collections.unmodifiableSet(topics); - } - /** * Get the Pattern to match all topics requiring to start reading from earliest available offset * @return the Pattern for matching all topics reading from earliest offset, never null */ public synchronized Pattern earliestResetTopicsPattern() { - Set topics = maybeDecorateInternalSourceTopics(earliestResetTopics); - - String[] sourceTopicNames = topics.toArray(new String[topics.size()]); - Pattern[] sourceTopicPatterns = earliestResetPatterns.toArray(new Pattern[earliestResetPatterns.size()]); - - Pattern earliestPattern = buildPatternForOffsetResetTopics(sourceTopicNames, sourceTopicPatterns); + final List topics = maybeDecorateInternalSourceTopics(earliestResetTopics); + final Pattern earliestPattern = buildPatternForOffsetResetTopics(topics, earliestResetPatterns); ensureNoRegexOverlap(earliestPattern, latestResetPatterns, latestResetTopics); @@ -1242,12 +1216,8 @@ public synchronized Pattern earliestResetTopicsPattern() { * @return the Pattern for matching all topics reading from latest offset, never null */ public synchronized Pattern latestResetTopicsPattern() { - Set topics = maybeDecorateInternalSourceTopics(latestResetTopics); - - String[] sourceTopicNames = topics.toArray(new String[topics.size()]); - Pattern[] sourceTopicPatterns = latestResetPatterns.toArray(new Pattern[latestResetPatterns.size()]); - - Pattern latestPattern = buildPatternForOffsetResetTopics(sourceTopicNames, sourceTopicPatterns); + final List topics = maybeDecorateInternalSourceTopics(latestResetTopics); + final Pattern latestPattern = buildPatternForOffsetResetTopics(topics, latestResetPatterns); ensureNoRegexOverlap(latestPattern, earliestResetPatterns, earliestResetTopics); @@ -1267,10 +1237,8 @@ private void ensureNoRegexOverlap(Pattern builtPattern, Set otherPatter throw new TopologyBuilderException(String.format("Found overlapping regex [%s] matching topic [%s] for a KStream with auto offset resets", builtPattern.pattern(), otherTopic)); } } - } - /** * Builds a composite pattern out of topic names and Pattern object for matching topic names. If the provided * arrays are empty a Pattern.compile("") instance is returned. @@ -1279,7 +1247,7 @@ private void ensureNoRegexOverlap(Pattern builtPattern, Set otherPatter * @param sourcePatterns Patterns for matching source topics to add to a composite pattern * @return a Pattern that is composed of the literal source topic names and any Patterns for matching source topics */ - private static synchronized Pattern buildPatternForOffsetResetTopics(String[] sourceTopics, Pattern[] sourcePatterns) { + private static synchronized Pattern buildPatternForOffsetResetTopics(Collection sourceTopics, Collection sourcePatterns) { StringBuilder builder = new StringBuilder(); for (String topic : sourceTopics) { @@ -1301,11 +1269,10 @@ private static synchronized Pattern buildPatternForOffsetResetTopics(String[] so /** * @return a mapping from state store name to a Set of source Topics. */ - public Map> stateStoreNameToSourceTopics() { - final Map> results = new HashMap<>(); + public Map> stateStoreNameToSourceTopics() { + final Map> results = new HashMap<>(); for (Map.Entry> entry : stateStoreNameToSourceTopics.entrySet()) { results.put(entry.getKey(), maybeDecorateInternalSourceTopics(entry.getValue())); - } return results; } @@ -1321,7 +1288,7 @@ public synchronized Collection> copartitionGroups() { for (Set nodeNames : copartitionSourceGroups) { Set copartitionGroup = new HashSet<>(); for (String node : nodeNames) { - String[] topics = nodeToSourceTopics.get(node); + final List topics = nodeToSourceTopics.get(node); if (topics != null) copartitionGroup.addAll(maybeDecorateInternalSourceTopics(topics)); } @@ -1330,12 +1297,8 @@ public synchronized Collection> copartitionGroups() { return Collections.unmodifiableList(list); } - private Set maybeDecorateInternalSourceTopics(final Set sourceTopics) { - return maybeDecorateInternalSourceTopics(sourceTopics.toArray(new String[sourceTopics.size()])); - } - - private Set maybeDecorateInternalSourceTopics(String ... sourceTopics) { - final Set decoratedTopics = new HashSet<>(); + private List maybeDecorateInternalSourceTopics(final Collection sourceTopics) { + final List decoratedTopics = new ArrayList<>(); for (String topic : sourceTopics) { if (internalTopicNames.contains(topic)) { decoratedTopics.add(decorateTopic(topic)); @@ -1357,23 +1320,18 @@ private String decorateTopic(String topic) { } public synchronized Pattern sourceTopicPattern() { - if (this.topicPattern == null && !nodeToSourcePatterns.isEmpty()) { - - List allNodeToSourceTopics = new ArrayList<>(); + if (this.topicPattern == null) { + List allSourceTopics = new ArrayList<>(); if (!nodeToSourceTopics.isEmpty()) { - for (String[] topics : nodeToSourceTopics.values()) { - allNodeToSourceTopics.addAll(Arrays.asList(topics)); - + for (List topics : nodeToSourceTopics.values()) { + allSourceTopics.addAll(maybeDecorateInternalSourceTopics(topics)); } } - int numPatterns = nodeToSourcePatterns.values().size(); - int numTopics = allNodeToSourceTopics.size(); - - Pattern[] patterns = nodeToSourcePatterns.values().toArray(new Pattern[numPatterns]); - String[] allTopics = allNodeToSourceTopics.toArray(new String[numTopics]); + Collections.sort(allSourceTopics); - this.topicPattern = buildPatternForOffsetResetTopics(allTopics, patterns); + this.topicPattern = buildPatternForOffsetResetTopics(allSourceTopics, nodeToSourcePatterns.values()); } + return this.topicPattern; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java index 0ebfda71c1e98..c4db740d56159 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java @@ -29,11 +29,11 @@ public class SinkNode extends ProcessorNode { private final String topic; private Serializer keySerializer; private Serializer valSerializer; - private final StreamPartitioner partitioner; + private final StreamPartitioner partitioner; private ProcessorContext context; - public SinkNode(String name, String topic, Serializer keySerializer, Serializer valSerializer, StreamPartitioner partitioner) { + public SinkNode(String name, String topic, Serializer keySerializer, Serializer valSerializer, StreamPartitioner partitioner) { super(name); this.topic = topic; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java index 771f5041a2f5a..340660640a323 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java @@ -21,14 +21,17 @@ import org.apache.kafka.streams.kstream.internals.ChangedDeserializer; import org.apache.kafka.streams.processor.ProcessorContext; +import java.util.List; + public class SourceNode extends ProcessorNode { + private final List topics; + + private ProcessorContext context; private Deserializer keyDeserializer; private Deserializer valDeserializer; - private ProcessorContext context; - private String[] topics; - public SourceNode(String name, String[] topics, Deserializer keyDeserializer, Deserializer valDeserializer) { + public SourceNode(String name, List topics, Deserializer keyDeserializer, Deserializer valDeserializer) { super(name); this.topics = topics; this.keyDeserializer = keyDeserializer; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 38961f23f617b..b445a51a48d6b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -180,14 +180,13 @@ private synchronized void setStateWhenNotInPendingShutdown(final State newState) protected final StreamsConfig config; protected final TopologyBuilder builder; - protected final Set sourceTopics; - protected final Pattern topicPattern; protected final Producer producer; protected final Consumer consumer; protected final Consumer restoreConsumer; private final String logPrefix; private final String threadClientId; + private final Pattern sourceTopicPattern; private final Map activeTasks; private final Map standbyTasks; private final Map activeTasksByPartition; @@ -200,6 +199,7 @@ private synchronized void setStateWhenNotInPendingShutdown(final State newState) private final long cleanTimeMs; private final long commitTimeMs; private final StreamsMetricsThreadImpl streamsMetrics; + // TODO: this is not private only for tests, should be better refactored final StateDirectory stateDirectory; private String originalReset; private StreamPartitionAssignor partitionAssignor = null; @@ -291,8 +291,7 @@ public StreamThread(TopologyBuilder builder, String threadName = getName(); this.config = config; this.builder = builder; - this.sourceTopics = builder.sourceTopics(); - this.topicPattern = builder.sourceTopicPattern(); + this.sourceTopicPattern = builder.sourceTopicPattern(); this.clientId = clientId; this.processId = processId; this.partitionGrouper = config.getConfiguredInstance(StreamsConfig.PARTITION_GROUPER_CLASS_CONFIG, PartitionGrouper.class); @@ -566,11 +565,7 @@ private void runLoop() { boolean requiresPoll = true; boolean polledRecords = false; - if (topicPattern != null) { - consumer.subscribe(topicPattern, rebalanceListener); - } else { - consumer.subscribe(new ArrayList<>(sourceTopics), rebalanceListener); - } + consumer.subscribe(sourceTopicPattern, rebalanceListener); while (stillRunning()) { this.timerStartedMs = time.milliseconds(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java index a59eb5f914f00..6fb6e062a4c84 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java @@ -84,7 +84,7 @@ public synchronized Collection getAllMetadataForStore(final Str return allMetadata; } - final Set sourceTopics = builder.stateStoreNameToSourceTopics().get(storeName); + final List sourceTopics = builder.stateStoreNameToSourceTopics().get(storeName); if (sourceTopics == null) { return Collections.emptyList(); } @@ -201,7 +201,7 @@ public synchronized void onChange(final Map> curre rebuildMetadata(currentState); } - private boolean hasPartitionsForAnyTopics(final Set topicNames, final Set partitionForHost) { + private boolean hasPartitionsForAnyTopics(final List topicNames, final Set partitionForHost) { for (TopicPartition topicPartition : partitionForHost) { if (topicNames.contains(topicPartition.topic())) { return true; @@ -215,13 +215,13 @@ private void rebuildMetadata(final Map> currentSta if (currentState.isEmpty()) { return; } - final Map> stores = builder.stateStoreNameToSourceTopics(); + final Map> stores = builder.stateStoreNameToSourceTopics(); for (Map.Entry> entry : currentState.entrySet()) { final HostInfo key = entry.getKey(); final Set partitionsForHost = new HashSet<>(entry.getValue()); final Set storesOnHost = new HashSet<>(); - for (Map.Entry> storeTopicEntry : stores.entrySet()) { - final Set topicsForStore = storeTopicEntry.getValue(); + for (Map.Entry> storeTopicEntry : stores.entrySet()) { + final List topicsForStore = storeTopicEntry.getValue(); if (hasPartitionsForAnyTopics(topicsForStore, partitionsForHost)) { storesOnHost.add(storeTopicEntry.getKey()); } @@ -259,7 +259,7 @@ private StreamsMetadata getStreamsMetadataForKey(final String storeName, } private SourceTopicsInfo getSourceTopicsInfo(final String storeName) { - final Set sourceTopics = builder.stateStoreNameToSourceTopics().get(storeName); + final List sourceTopics = builder.stateStoreNameToSourceTopics().get(storeName); if (sourceTopics == null || sourceTopics.isEmpty()) { return null; } @@ -271,11 +271,11 @@ private boolean isInitialized() { } private class SourceTopicsInfo { - private final Set sourceTopics; + private final List sourceTopics; private int maxPartitions; private String topicWithMostPartitions; - private SourceTopicsInfo(final Set sourceTopics) { + private SourceTopicsInfo(final List sourceTopics) { this.sourceTopics = sourceTopics; for (String topic : sourceTopics) { final List partitions = clusterMetadata.partitionsForTopic(topic); diff --git a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java index 41277c7f264b4..5dae8ddf580ad 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.config.ConfigException; @@ -201,6 +202,7 @@ public void shouldReturnFalseOnCloseWhenThreadsHaventTerminated() throws Excepti final Properties props = new Properties(); props.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "appId"); props.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); final KStreamBuilder builder = new KStreamBuilder(); final CountDownLatch latch = new CountDownLatch(1); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java index 6f3c95a3ab536..f0fb0a2a4673d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java @@ -354,9 +354,9 @@ private void createTopics() throws InterruptedException { streamOneInput = "stream-one-" + testNo; streamTwoInput = "stream-two-" + testNo; streamFourInput = "stream-four-" + testNo; - CLUSTER.createTopic(streamOneInput); - CLUSTER.createTopic(streamTwoInput); - CLUSTER.createTopic(streamFourInput); + CLUSTER.createTopic(streamOneInput, 2, 1); + CLUSTER.createTopic(streamTwoInput, 2, 1); + CLUSTER.createTopic(streamFourInput, 2, 1); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java index 619b6b5ed87e3..fe7bebc5635a6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java @@ -68,7 +68,7 @@ public void start() throws IOException, InterruptedException { putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.DeleteTopicEnableProp(), true); putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.LogCleanerDedupeBufferSizeProp(), 2 * 1024 * 1024L); putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.GroupMinSessionTimeoutMsProp(), 0); - putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), false); + putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), true); for (int i = 0; i < brokers.length; i++) { brokerConfig.put(KafkaConfig$.MODULE$.BrokerIdProp(), i); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java index 875c3591b881b..3e7c41b347f22 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java @@ -214,7 +214,7 @@ public boolean conditionMet() { } }; - final String conditionDetails = "Did not receive " + expectedNumRecords + " number of records"; + final String conditionDetails = "Expecting " + expectedNumRecords + " records while only received " + accumData.size() + ": " + accumData; TestUtils.waitForCondition(valuesRead, waitTime, conditionDetails); @@ -254,7 +254,7 @@ public boolean conditionMet() { } }; - final String conditionDetails = "Did not receive " + expectedNumRecords + " number of records"; + final String conditionDetails = "Expecting " + expectedNumRecords + " records while only received " + accumData.size() + ": " + accumData; TestUtils.waitForCondition(valuesRead, waitTime, conditionDetails); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java index 5f126c33e9d4f..a469f25fbe754 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java @@ -133,9 +133,8 @@ public boolean test(final String key, final String value) { final KStream merged = builder.merge(processedSource1, processedSource2, source3); merged.groupByKey().count("my-table"); - final Map> actual = builder.stateStoreNameToSourceTopics(); - - assertEquals(Utils.mkSet("topic-1", "topic-2", "topic-3"), actual.get("my-table")); + final Map> actual = builder.stateStoreNameToSourceTopics(); + assertEquals(Utils.mkList("topic-1", "topic-2", "topic-3"), actual.get("my-table")); } @Test(expected = TopologyBuilderException.class) @@ -227,12 +226,11 @@ public void shouldMapStateStoresToCorrectSourceTopics() throws Exception { final KStream playEvents = builder.stream("events"); final KTable table = builder.table("table-topic", "table-store"); - assertEquals(Collections.singleton("table-topic"), builder.stateStoreNameToSourceTopics().get("table-store")); + assertEquals(Collections.singletonList("table-topic"), builder.stateStoreNameToSourceTopics().get("table-store")); final KStream mapped = playEvents.map(MockKeyValueMapper.SelectValueKeyValueMapper()); mapped.leftJoin(table, MockValueJoiner.TOSTRING_JOINER).groupByKey().count("count"); - - assertEquals(Collections.singleton("table-topic"), builder.stateStoreNameToSourceTopics().get("table-store")); - assertEquals(Collections.singleton(APP_ID + "-KSTREAM-MAP-0000000003-repartition"), builder.stateStoreNameToSourceTopics().get("count")); + assertEquals(Collections.singletonList("table-topic"), builder.stateStoreNameToSourceTopics().get("table-store")); + assertEquals(Collections.singletonList(APP_ID + "-KSTREAM-MAP-0000000003-repartition"), builder.stateStoreNameToSourceTopics().get("count")); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java index 4712320d160ed..2f3a450f811ee 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java @@ -156,12 +156,9 @@ public void testSourceTopics() { builder.addSource("source-3", "topic-3"); builder.addInternalTopic("topic-3"); - Set expected = new HashSet<>(); - expected.add("topic-1"); - expected.add("topic-2"); - expected.add("X-topic-3"); + Pattern expectedPattern = Pattern.compile("X-topic-3|topic-1|topic-2"); - assertEquals(expected, builder.sourceTopics()); + assertEquals(expectedPattern.pattern(), builder.sourceTopicPattern().pattern()); } @Test @@ -184,7 +181,7 @@ public void testAddMoreThanOnePatternSourceNode() { @Test public void testSubscribeTopicNameAndPattern() { final TopologyBuilder builder = new TopologyBuilder(); - Pattern expectedPattern = Pattern.compile("topic-foo|topic-bar|.*-\\d"); + Pattern expectedPattern = Pattern.compile("topic-bar|topic-foo|.*-\\d"); builder.addSource("source-1", "topic-foo", "topic-bar"); builder.addSource("source-2", Pattern.compile(".*-\\d")); assertEquals(expectedPattern.pattern(), builder.sourceTopicPattern().pattern()); @@ -441,9 +438,9 @@ public void shouldAssociateStateStoreNameWhenStateStoreSupplierIsInternal() thro builder.addSource("source", "topic"); builder.addProcessor("processor", new MockProcessorSupplier(), "source"); builder.addStateStore(new MockStateStoreSupplier("store", false), "processor"); - final Map> stateStoreNameToSourceTopic = builder.stateStoreNameToSourceTopics(); + final Map> stateStoreNameToSourceTopic = builder.stateStoreNameToSourceTopics(); assertEquals(1, stateStoreNameToSourceTopic.size()); - assertEquals(Collections.singleton("topic"), stateStoreNameToSourceTopic.get("store")); + assertEquals(Collections.singletonList("topic"), stateStoreNameToSourceTopic.get("store")); } @Test @@ -452,9 +449,9 @@ public void shouldAssociateStateStoreNameWhenStateStoreSupplierIsExternal() thro builder.addSource("source", "topic"); builder.addProcessor("processor", new MockProcessorSupplier(), "source"); builder.addStateStore(new MockStateStoreSupplier("store", false), "processor"); - final Map> stateStoreNameToSourceTopic = builder.stateStoreNameToSourceTopics(); + final Map> stateStoreNameToSourceTopic = builder.stateStoreNameToSourceTopics(); assertEquals(1, stateStoreNameToSourceTopic.size()); - assertEquals(Collections.singleton("topic"), stateStoreNameToSourceTopic.get("store")); + assertEquals(Collections.singletonList("topic"), stateStoreNameToSourceTopic.get("store")); } @Test @@ -465,9 +462,9 @@ public void shouldCorrectlyMapStateStoreToInternalTopics() throws Exception { builder.addSource("source", "internal-topic"); builder.addProcessor("processor", new MockProcessorSupplier(), "source"); builder.addStateStore(new MockStateStoreSupplier("store", false), "processor"); - final Map> stateStoreNameToSourceTopic = builder.stateStoreNameToSourceTopics(); + final Map> stateStoreNameToSourceTopic = builder.stateStoreNameToSourceTopics(); assertEquals(1, stateStoreNameToSourceTopic.size()); - assertEquals(Collections.singleton("appId-internal-topic"), stateStoreNameToSourceTopic.get("store")); + assertEquals(Collections.singletonList("appId-internal-topic"), stateStoreNameToSourceTopic.get("store")); } @SuppressWarnings("unchecked") diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializerTest.java index fdd9127cd2cb4..cf328eee808ca 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializerTest.java @@ -22,6 +22,8 @@ import org.apache.kafka.streams.errors.StreamsException; import org.junit.Test; +import java.util.Collections; + import static org.junit.Assert.assertEquals; public class SourceNodeRecordDeserializerTest { @@ -82,7 +84,7 @@ static class TheSourceNode extends SourceNode { final boolean valueThrowsException, final Object key, final Object value) { - super("", new String[0], null, null); + super("", Collections.EMPTY_LIST, null, null); this.keyThrowsException = keyThrowsException; this.valueThrowsException = valueThrowsException; this.key = key; diff --git a/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java index 096c64a6c069c..4e0d21a062e69 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java +++ b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java @@ -23,6 +23,7 @@ import org.apache.kafka.streams.processor.internals.SourceNode; import java.util.ArrayList; +import java.util.Arrays; import java.util.concurrent.atomic.AtomicInteger; public class MockSourceNode extends SourceNode { @@ -36,7 +37,7 @@ public class MockSourceNode extends SourceNode { public boolean initialized; public MockSourceNode(String[] topics, Deserializer keyDeserializer, Deserializer valDeserializer) { - super(NAME + INDEX.getAndIncrement(), topics, keyDeserializer, valDeserializer); + super(NAME + INDEX.getAndIncrement(), Arrays.asList(topics), keyDeserializer, valDeserializer); } @Override From 4218c02478a213935afbb2c2d661d8a29a5f29ee Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Mon, 23 Jan 2017 16:31:36 -0800 Subject: [PATCH 028/177] KAFKA-4671: Fix Streams window retention policy Author: Matthias J. Sax Reviewers: Damian Guy, Eno Thereska, Guozhang Wang Closes #2401 from mjsax/kafka-4671-window-retention-policy (cherry picked from commit 79987590e3e96351ff75ce86718801ec605b2419) Signed-off-by: Guozhang Wang --- .../kafka/streams/kstream/JoinWindows.java | 60 +++++---- .../kafka/streams/kstream/SessionWindows.java | 19 ++- .../kafka/streams/kstream/TimeWindows.java | 68 +++++----- .../streams/kstream/UnlimitedWindows.java | 38 +++--- .../apache/kafka/streams/kstream/Window.java | 42 +++--- .../kafka/streams/kstream/Windowed.java | 15 +-- .../apache/kafka/streams/kstream/Windows.java | 24 ++-- .../kstream/internals/KStreamImpl.java | 12 +- .../kstream/internals/SessionWindow.java | 6 +- .../streams/kstream/internals/TimeWindow.java | 11 +- .../kstream/internals/UnlimitedWindow.java | 12 +- .../streams/kstream/JoinWindowsTest.java | 64 +++++++-- .../streams/kstream/SessionWindowsTest.java | 68 ++++++++++ .../streams/kstream/TimeWindowsTest.java | 92 ++++++++++--- .../streams/kstream/UnlimitedWindowsTest.java | 22 +++- .../kafka/streams/kstream/WindowTest.java | 85 ++++++++++++ .../kafka/streams/kstream/WindowsTest.java | 62 +++++++++ .../kstream/internals/TimeWindowTest.java | 122 ++++++++++++++++++ .../internals/UnlimitedWindowTest.java | 42 ++++++ .../WindowedStreamPartitionerTest.java | 2 +- .../CompositeReadOnlySessionStoreTest.java | 18 +-- 21 files changed, 701 insertions(+), 183 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/WindowTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindowTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java index 931774356e976..6dd1a85f5340a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java @@ -17,8 +17,6 @@ package org.apache.kafka.streams.kstream; -import org.apache.kafka.streams.kstream.internals.TimeWindow; - import java.util.Map; /** @@ -45,21 +43,19 @@ * Both values (before and after) must not result in an "inverse" window, * i.e., lower-interval-bound must not be larger than upper-interval.bound. */ -public class JoinWindows extends Windows { +public class JoinWindows extends Windows { /** Maximum time difference for tuples that are before the join tuple. */ - public final long before; + public final long beforeMs; /** Maximum time difference for tuples that are after the join tuple. */ - public final long after; - - private JoinWindows(long before, long after) { - super(); + public final long afterMs; - if (before + after < 0) { - throw new IllegalArgumentException("Window interval (ie, before+after) must not be negative"); + private JoinWindows(final long beforeMs, final long afterMs) { + if (beforeMs + afterMs < 0) { + throw new IllegalArgumentException("Window interval (ie, beforeMs+afterMs) must not be negative"); } - this.after = after; - this.before = before; + this.afterMs = afterMs; + this.beforeMs = beforeMs; } /** @@ -68,8 +64,8 @@ private JoinWindows(long before, long after) { * * @param timeDifference join window interval */ - public static JoinWindows of(long timeDifference) { - return new JoinWindows(timeDifference, timeDifference); + public static JoinWindows of(final long timeDifferenceMs) throws IllegalArgumentException { + return new JoinWindows(timeDifferenceMs, timeDifferenceMs); } /** @@ -79,8 +75,8 @@ public static JoinWindows of(long timeDifference) { * * @param timeDifference join window interval */ - public JoinWindows before(long timeDifference) { - return new JoinWindows(timeDifference, this.after); + public JoinWindows before(final long timeDifferenceMs) throws IllegalArgumentException { + return new JoinWindows(timeDifferenceMs, afterMs); } /** @@ -90,25 +86,39 @@ public JoinWindows before(long timeDifference) { * * @param timeDifference join window interval */ - public JoinWindows after(long timeDifference) { - return new JoinWindows(this.before, timeDifference); + public JoinWindows after(final long timeDifferenceMs) throws IllegalArgumentException { + return new JoinWindows(beforeMs, timeDifferenceMs); } /** * Not supported by {@link JoinWindows}. Throws {@link UnsupportedOperationException}. */ @Override - public Map windowsFor(long timestamp) { + public Map windowsFor(final long timestamp) { throw new UnsupportedOperationException("windowsFor() is not supported in JoinWindows"); } @Override public long size() { - return after + before; + return beforeMs + afterMs; + } + + @Override + public JoinWindows until(final long durationMs) throws IllegalArgumentException { + if (durationMs < size()) { + throw new IllegalArgumentException("Window retention time (durationMs) cannot be smaller than the window size."); + } + super.until(durationMs); + return this; + } + + @Override + public long maintainMs() { + return Math.max(super.maintainMs(), size()); } @Override - public final boolean equals(Object o) { + public final boolean equals(final Object o) { if (o == this) { return true; } @@ -116,14 +126,14 @@ public final boolean equals(Object o) { return false; } - JoinWindows other = (JoinWindows) o; - return this.before == other.before && this.after == other.after; + final JoinWindows other = (JoinWindows) o; + return beforeMs == other.beforeMs && afterMs == other.afterMs; } @Override public int hashCode() { - int result = (int) (before ^ (before >>> 32)); - result = 31 * result + (int) (after ^ (after >>> 32)); + int result = (int) (beforeMs ^ (beforeMs >>> 32)); + result = 31 * result + (int) (afterMs ^ (afterMs >>> 32)); return result; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java index f9a399a6451e1..bed6c3dd97125 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java @@ -63,9 +63,9 @@ public class SessionWindows { private final long gapMs; private long maintainDurationMs; - private SessionWindows(final long gapMs, final long maintainDurationMs) { + private SessionWindows(final long gapMs) { this.gapMs = gapMs; - this.maintainDurationMs = maintainDurationMs; + maintainDurationMs = Windows.DEFAULT_MAINTAIN_DURATION_MS; } /** @@ -75,7 +75,10 @@ private SessionWindows(final long gapMs, final long maintainDurationMs) { * and default maintain duration */ public static SessionWindows with(final long inactivityGapMs) { - return new SessionWindows(inactivityGapMs, Windows.DEFAULT_MAINTAIN_DURATION); + if (inactivityGapMs < 1) { + throw new IllegalArgumentException("Gap time (inactivityGapMs) cannot be zero or negative."); + } + return new SessionWindows(inactivityGapMs); } /** @@ -84,8 +87,12 @@ public static SessionWindows with(final long inactivityGapMs) { * * @return itself */ - public SessionWindows until(final long durationMs) { - this.maintainDurationMs = durationMs; + public SessionWindows until(final long durationMs) throws IllegalArgumentException { + if (durationMs < gapMs) { + throw new IllegalArgumentException("Window retentin time (durationMs) cannot be smaller than window gap."); + } + maintainDurationMs = durationMs; + return this; } @@ -100,6 +107,6 @@ public long inactivityGap() { * @return the minimum amount of time a window will be maintained for. */ public long maintainMs() { - return maintainDurationMs; + return Math.max(maintainDurationMs, gapMs); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java index ef94cf9b0abfa..11df22823c6aa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java @@ -40,27 +40,18 @@ public class TimeWindows extends Windows { * The window size's effective time unit is determined by the semantics of the topology's * configured {@link org.apache.kafka.streams.processor.TimestampExtractor}. */ - public final long size; + public final long sizeMs; /** * The size of the window's advance interval, i.e. by how much a window moves forward relative * to the previous one. The interval's effective time unit is determined by the semantics of * the topology's configured {@link org.apache.kafka.streams.processor.TimestampExtractor}. */ - public final long advance; + public final long advanceMs; - - private TimeWindows(long size, long advance) { - super(); - if (size <= 0) { - throw new IllegalArgumentException("window size must be > 0 (you provided " + size + ")"); - } - this.size = size; - if (!(0 < advance && advance <= size)) { - throw new IllegalArgumentException( - String.format("advance interval (%d) must lie within interval (0, %d]", advance, size)); - } - this.advance = advance; + private TimeWindows(final long sizeMs, final long advanceMs) throws IllegalArgumentException { + this.sizeMs = sizeMs; + this.advanceMs = advanceMs; } /** @@ -76,8 +67,11 @@ private TimeWindows(long size, long advance) { * topology's configured {@link org.apache.kafka.streams.processor.TimestampExtractor}. * @return a new window definition */ - public static TimeWindows of(long size) { - return new TimeWindows(size, size); + public static TimeWindows of(final long sizeMs) throws IllegalArgumentException { + if (sizeMs <= 0) { + throw new IllegalArgumentException("Window sizeMs must be larger than zero."); + } + return new TimeWindows(sizeMs, sizeMs); } /** @@ -93,43 +87,59 @@ public static TimeWindows of(long size) { * {@link org.apache.kafka.streams.processor.TimestampExtractor}. * @return a new window definition */ - public TimeWindows advanceBy(long interval) { - return new TimeWindows(this.size, interval); + public TimeWindows advanceBy(final long advanceMs) { + if (advanceMs <= 0 || advanceMs > sizeMs) { + throw new IllegalArgumentException(String.format("AdvanceMs must lie within interval (0, %d]", sizeMs)); + } + return new TimeWindows(sizeMs, advanceMs); } @Override - public Map windowsFor(long timestamp) { - long windowStart = (Math.max(0, timestamp - this.size + this.advance) / this.advance) * this.advance; - Map windows = new HashMap<>(); + public Map windowsFor(final long timestamp) { + long windowStart = (Math.max(0, timestamp - sizeMs + advanceMs) / advanceMs) * advanceMs; + final Map windows = new HashMap<>(); while (windowStart <= timestamp) { - TimeWindow window = new TimeWindow(windowStart, windowStart + this.size); + final TimeWindow window = new TimeWindow(windowStart, windowStart + sizeMs); windows.put(windowStart, window); - windowStart += this.advance; + windowStart += advanceMs; } return windows; } @Override public long size() { - return size; + return sizeMs; + } + + public TimeWindows until(final long durationMs) throws IllegalArgumentException { + if (durationMs < sizeMs) { + throw new IllegalArgumentException("Window retention time (durationMs) cannot be smaller than the window size."); + } + super.until(durationMs); + return this; + } + + @Override + public long maintainMs() { + return Math.max(super.maintainMs(), sizeMs); } @Override - public final boolean equals(Object o) { + public final boolean equals(final Object o) { if (o == this) { return true; } if (!(o instanceof TimeWindows)) { return false; } - TimeWindows other = (TimeWindows) o; - return this.size == other.size && this.advance == other.advance; + final TimeWindows other = (TimeWindows) o; + return sizeMs == other.sizeMs && advanceMs == other.advanceMs; } @Override public int hashCode() { - int result = (int) (size ^ (size >>> 32)); - result = 31 * result + (int) (advance ^ (advance >>> 32)); + int result = (int) (sizeMs ^ (sizeMs >>> 32)); + result = 31 * result + (int) (advanceMs ^ (advanceMs >>> 32)); return result; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java index 3dc6f654dcdec..8605f9d6bf26b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java @@ -27,24 +27,23 @@ */ public class UnlimitedWindows extends Windows { - private static final long DEFAULT_START_TIMESTAMP = 0L; + private static final long DEFAULT_START_TIMESTAMP_MS = 0L; /** The start timestamp of the window. */ - public final long start; + public final long startMs; - private UnlimitedWindows(long start) { - super(); - if (start < 0) { - throw new IllegalArgumentException("start must be > 0 (you provided " + start + ")"); + private UnlimitedWindows(final long startMs) throws IllegalArgumentException { + if (startMs < 0) { + throw new IllegalArgumentException("startMs must be > 0 (you provided " + startMs + ")"); } - this.start = start; + this.startMs = startMs; } /** * Return an unlimited window starting at timestamp zero. */ public static UnlimitedWindows of() { - return new UnlimitedWindows(DEFAULT_START_TIMESTAMP); + return new UnlimitedWindows(DEFAULT_START_TIMESTAMP_MS); } /** @@ -53,18 +52,18 @@ public static UnlimitedWindows of() { * @param start the window start time * @return a new unlimited window that starts at {@code start} */ - public UnlimitedWindows startOn(long start) { + public UnlimitedWindows startOn(final long start) throws IllegalArgumentException { return new UnlimitedWindows(start); } @Override - public Map windowsFor(long timestamp) { + public Map windowsFor(final long timestamp) { // always return the single unlimited window // we cannot use Collections.singleMap since it does not support remove() - Map windows = new HashMap<>(); - if (timestamp >= start) { - windows.put(start, new UnlimitedWindow(start)); + final Map windows = new HashMap<>(); + if (timestamp >= startMs) { + windows.put(startMs, new UnlimitedWindow(startMs)); } return windows; } @@ -75,7 +74,7 @@ public long size() { } @Override - public final boolean equals(Object o) { + public final boolean equals(final Object o) { if (o == this) { return true; } @@ -84,13 +83,18 @@ public final boolean equals(Object o) { return false; } - UnlimitedWindows other = (UnlimitedWindows) o; - return this.start == other.start; + final UnlimitedWindows other = (UnlimitedWindows) o; + return startMs == other.startMs; } @Override public int hashCode() { - return (int) (start ^ (start >>> 32)); + return (int) (startMs ^ (startMs >>> 32)); + } + + @Override + public UnlimitedWindows until(final long durationMs) { + throw new IllegalArgumentException("Window retention time (durationMs) cannot be set for UnlimitedWindows."); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java index 13a9529d6bedd..9c6edc099588c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java @@ -22,8 +22,8 @@ */ public abstract class Window { - protected final long start; - protected final long end; + protected final long startMs; + protected final long endMs; /** * Create a new window for the given start time (inclusive) and end time (exclusive). @@ -33,32 +33,29 @@ public abstract class Window { * @throws IllegalArgumentException if {@code start} or {@code end} is negative or if {@code end} is smaller than * {@code start} */ - public Window(long start, long end) throws IllegalArgumentException { - if (start < 0) { - throw new IllegalArgumentException("Window start time cannot be negative."); + public Window(long startMs, long endMs) throws IllegalArgumentException { + if (startMs < 0) { + throw new IllegalArgumentException("Window startMs time cannot be negative."); } - if (end < 0) { - throw new IllegalArgumentException("Window end time cannot be negative."); + if (endMs < startMs) { + throw new IllegalArgumentException("Window endMs time cannot be smaller than window startMs time."); } - if (end < start) { - throw new IllegalArgumentException("Window end time cannot be smaller than window start time."); - } - this.start = start; - this.end = end; + this.startMs = startMs; + this.endMs = endMs; } /** * Return the start timestamp of this window, inclusive */ public long start() { - return start; + return startMs; } /** * Return the end timestamp of this window, exclusive */ public long end() { - return end; + return endMs; } /** @@ -67,10 +64,10 @@ public long end() { * @param other another window * @return {@code true} if {@code other} overlaps with this window—{@code false} otherwise */ - public abstract boolean overlap(Window other); + public abstract boolean overlap(final Window other); @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { if (obj == this) { return true; } @@ -79,21 +76,20 @@ public boolean equals(Object obj) { return false; } - Window other = (Window) obj; - return this.start == other.start && this.end == other.end; + final Window other = (Window) obj; + return startMs == other.startMs && endMs == other.endMs; } @Override public int hashCode() { - long n = (this.start << 32) | this.end; - return (int) (n % 0xFFFFFFFFL); + return (int) (((startMs << 32) | endMs) % 0xFFFFFFFFL); } @Override public String toString() { return "Window{" + - "start=" + start + - ", end=" + end + - '}'; + "start=" + startMs + + ", end=" + endMs + + '}'; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java index 6606fcb367acc..81357c1daf50d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java @@ -26,11 +26,11 @@ */ public class Windowed { - private K key; + private final K key; - private Window window; + private final Window window; - public Windowed(K key, Window window) { + public Windowed(final K key, final Window window) { this.key = key; this.window = window; } @@ -59,21 +59,20 @@ public String toString() { } @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { if (obj == this) return true; if (!(obj instanceof Windowed)) return false; - Windowed that = (Windowed) obj; - - return this.window.equals(that.window) && this.key.equals(that.key); + final Windowed that = (Windowed) obj; + return window.equals(that.window) && key.equals(that.key); } @Override public int hashCode() { - long n = ((long) window.hashCode() << 32) | key.hashCode(); + final long n = ((long) window.hashCode() << 32) | key.hashCode(); return (int) (n % 0xFFFFFFFFL); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java index ebd92fe7346e5..29b61fda79bf8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream; import java.util.Map; @@ -28,15 +27,15 @@ public abstract class Windows { private static final int DEFAULT_NUM_SEGMENTS = 3; - static final long DEFAULT_MAINTAIN_DURATION = 24 * 60 * 60 * 1000L; // one day + static final long DEFAULT_MAINTAIN_DURATION_MS = 24 * 60 * 60 * 1000L; // one day private long maintainDurationMs; public int segments; protected Windows() { - this.segments = DEFAULT_NUM_SEGMENTS; - this.maintainDurationMs = DEFAULT_MAINTAIN_DURATION; + segments = DEFAULT_NUM_SEGMENTS; + maintainDurationMs = DEFAULT_MAINTAIN_DURATION_MS; } /** @@ -45,8 +44,12 @@ protected Windows() { * * @return itself */ - public Windows until(long durationMs) { - this.maintainDurationMs = durationMs; + // This should always get overridden to provide the correct return type and thus to avoid a cast + public Windows until(final long durationMs) throws IllegalArgumentException { + if (durationMs < 0) { + throw new IllegalArgumentException("Window retention time (durationMs) cannot be negative."); + } + maintainDurationMs = durationMs; return this; } @@ -57,7 +60,10 @@ public Windows until(long durationMs) { * * @return itself */ - protected Windows segments(int segments) { + protected Windows segments(final int segments) throws IllegalArgumentException { + if (segments < 2) { + throw new IllegalArgumentException("Number of segments must be at least 2."); + } this.segments = segments; return this; @@ -69,7 +75,7 @@ protected Windows segments(int segments) { * @return the window maintain duration in milliseconds of streams time */ public long maintainMs() { - return this.maintainDurationMs; + return maintainDurationMs; } /** @@ -78,7 +84,7 @@ public long maintainMs() { * @param timestamp the timestamp window should get created for * @return a map of {@code windowStartTimestamp -> Window} entries */ - public abstract Map windowsFor(long timestamp); + public abstract Map windowsFor(final long timestamp); public abstract long size(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 3d41ae44d768d..0434f06c7fe76 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -711,20 +711,20 @@ public KStream join(KStream lhs, KStreamJoinWindow thisWindowedStream = new KStreamJoinWindow<>(thisWindow.name(), - windows.before + windows.after + 1, + windows.beforeMs + windows.afterMs + 1, windows.maintainMs()); KStreamJoinWindow otherWindowedStream = new KStreamJoinWindow<>(otherWindow.name(), - windows.before + windows.after + 1, + windows.beforeMs + windows.afterMs + 1, windows.maintainMs()); final KStreamKStreamJoin joinThis = new KStreamKStreamJoin<>(otherWindow.name(), - windows.before, - windows.after, + windows.beforeMs, + windows.afterMs, joiner, leftOuter); final KStreamKStreamJoin joinOther = new KStreamKStreamJoin<>(thisWindow.name(), - windows.after, - windows.before, + windows.afterMs, + windows.beforeMs, reverseJoiner(joiner), rightOuter); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindow.java index db63029e04ae7..cf72752fa526b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindow.java @@ -35,8 +35,8 @@ public final class SessionWindow extends Window { * @param start the start timestamp of the window * @param end the end timestamp of the window */ - public SessionWindow(final long start, final long end) { - super(start, end); + public SessionWindow(final long startMs, final long endMs) { + super(startMs, endMs); } /** @@ -52,7 +52,7 @@ public boolean overlap(final Window other) throws IllegalArgumentException { + other.getClass()); } final SessionWindow otherWindow = (SessionWindow) other; - return !(otherWindow.end < start || end < otherWindow.start); + return !(otherWindow.endMs < startMs || endMs < otherWindow.startMs); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java index 630821fdb8064..bf98f941dbac9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java @@ -21,18 +21,21 @@ public class TimeWindow extends Window { - public TimeWindow(long start, long end) { - super(start, end); + public TimeWindow(long startMs, long endMs) { + super(startMs, endMs); + if (startMs == endMs) { + throw new IllegalArgumentException("Window endMs must be greater than window startMs."); + } } @Override - public boolean overlap(Window other) { + public boolean overlap(final Window other) throws IllegalArgumentException { if (getClass() != other.getClass()) { throw new IllegalArgumentException("Cannot compare windows of different type. Other window has type " + other.getClass()); } final TimeWindow otherWindow = (TimeWindow) other; - return start < otherWindow.end && otherWindow.start < end; + return startMs < otherWindow.endMs && otherWindow.startMs < endMs; } } \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java index e9ec040d0d191..7fb7c53000a44 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java @@ -5,9 +5,9 @@ * 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 - * + *

      + * 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. @@ -21,12 +21,12 @@ public class UnlimitedWindow extends Window { - public UnlimitedWindow(long start) { - super(start, Long.MAX_VALUE); + public UnlimitedWindow(final long startMs) { + super(startMs, Long.MAX_VALUE); } @Override - public boolean overlap(Window other) { + public boolean overlap(final Window other) { if (getClass() != other.getClass()) { throw new IllegalArgumentException("Cannot compare windows of different type. Other window has type " + other.getClass()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java index b37e5e8d87986..24387ad687805 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.fail; public class JoinWindowsTest { @@ -44,8 +45,8 @@ public void shouldHaveSaneEqualsAndHashCode() { assertEquals(w2, w1); assertEquals(w1.hashCode(), w2.hashCode()); - JoinWindows w3 = JoinWindows.of(w2.after).before(anyOtherSize); - JoinWindows w4 = JoinWindows.of(anyOtherSize).after(w2.after); + JoinWindows w3 = JoinWindows.of(w2.afterMs).before(anyOtherSize); + JoinWindows w4 = JoinWindows.of(anyOtherSize).after(w2.afterMs); assertEquals(w3, w4); assertEquals(w4, w3); assertEquals(w3.hashCode(), w4.hashCode()); @@ -55,13 +56,13 @@ public void shouldHaveSaneEqualsAndHashCode() { assertNotEquals("must be false for different window types", UnlimitedWindows.of(), w1); assertNotEquals("must be false for different types", new Object(), w1); - JoinWindows differentWindowSize = JoinWindows.of(w1.after + 1); + JoinWindows differentWindowSize = JoinWindows.of(w1.afterMs + 1); assertNotEquals("must be false when window sizes are different", differentWindowSize, w1); - JoinWindows differentWindowSize2 = JoinWindows.of(w1.after).after(w1.after + 1); + JoinWindows differentWindowSize2 = JoinWindows.of(w1.afterMs).after(w1.afterMs + 1); assertNotEquals("must be false when window sizes are different", differentWindowSize2, w1); - JoinWindows differentWindowSize3 = JoinWindows.of(w1.after).before(w1.before + 1); + JoinWindows differentWindowSize3 = JoinWindows.of(w1.afterMs).before(w1.beforeMs + 1); assertNotEquals("must be false when window sizes are different", differentWindowSize3, w1); } @@ -85,14 +86,55 @@ public void timeDifferenceMustNotBeNegative() { JoinWindows.of(-1); } - @Test(expected = IllegalArgumentException.class) - public void afterBelowLower() { - JoinWindows.of(anySize).after(-anySize - 1); + @Test + public void endTimeShouldNotBeBeforeStart() { + final JoinWindows windowSpec = JoinWindows.of(anySize); + try { + windowSpec.after(-anySize - 1); + fail("window end time should not be before window start time"); + } catch (final IllegalArgumentException e) { + // expected + } } - @Test(expected = IllegalArgumentException.class) - public void beforeOverUpper() { - JoinWindows.of(anySize).before(-anySize - 1); + @Test + public void startTimeShouldNotBeAfterEnd() { + final JoinWindows windowSpec = JoinWindows.of(anySize); + try { + windowSpec.before(-anySize - 1); + fail("window start time should not be after window end time"); + } catch (final IllegalArgumentException e) { + // expected + } + } + + @Test + public void untilShouldSetMaintainDuration() { + final JoinWindows windowSpec = JoinWindows.of(anySize); + final long windowSize = windowSpec.size(); + assertEquals(windowSize, windowSpec.until(windowSize).maintainMs()); + } + + @Test + public void shouldUseWindowSizeForMaintainDurationWhenSizeLargerThanDefaultMaintainMs() { + final long size = Windows.DEFAULT_MAINTAIN_DURATION_MS; + + final JoinWindows windowSpec = JoinWindows.of(size); + final long windowSize = windowSpec.size(); + + assertEquals(windowSize, windowSpec.maintainMs()); + } + + @Test + public void retentionTimeMustNoBeSmallerThanWindowSize() { + final JoinWindows windowSpec = JoinWindows.of(anySize); + final long windowSize = windowSpec.size(); + try { + windowSpec.until(windowSize - 1); + fail("should not accept retention time smaller than window size"); + } catch (final IllegalArgumentException e) { + // expected + } } } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java new file mode 100644 index 0000000000000..a9eced46cf1a3 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.kafka.streams.kstream; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +public class SessionWindowsTest { + + @Test + public void shouldSetWindowGap() { + final long anyGap = 42L; + assertEquals(anyGap, SessionWindows.with(anyGap).inactivityGap()); + } + + @Test + public void shouldSetWindowRetentionTime() { + final long anyRetentionTime = 42L; + assertEquals(anyRetentionTime, SessionWindows.with(1).until(anyRetentionTime).maintainMs()); + } + + @Test(expected = IllegalArgumentException.class) + public void windowSizeMustNotBeNegative() { + SessionWindows.with(-1); + } + + @Test(expected = IllegalArgumentException.class) + public void windowSizeMustNotBeZero() { + SessionWindows.with(0); + } + + @Test + public void retentionTimeShouldBeGapIfGapIsLargerThanDefaultRetentionTime() { + final long windowGap = 2 * Windows.DEFAULT_MAINTAIN_DURATION_MS; + assertEquals(windowGap, SessionWindows.with(windowGap).maintainMs()); + } + + @Test + public void retentionTimeMustNotBeNegative() { + final SessionWindows windowSpec = SessionWindows.with(42); + try { + windowSpec.until(41); + fail("should not accept retention time smaller than gap"); + } catch (final IllegalArgumentException e) { + // expected + } + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java index 2bea16b23b367..6b8b6ea9c99c1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java @@ -26,15 +26,38 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.fail; public class TimeWindowsTest { - private static long anySize = 123L; + private static final long ANY_SIZE = 123L; + + @Test + public void shouldSetWindowSize() { + assertEquals(ANY_SIZE, TimeWindows.of(ANY_SIZE).sizeMs); + } + + @Test + public void shouldSetWindowAdvance() { + final long anyAdvance = 4; + assertEquals(anyAdvance, TimeWindows.of(ANY_SIZE).advanceBy(anyAdvance).advanceMs); + } + + @Test + public void shouldSetWindowRetentionTime() { + assertEquals(ANY_SIZE, TimeWindows.of(ANY_SIZE).until(ANY_SIZE).maintainMs()); + } + + @Test + public void shouldUseWindowSizeAsRentitionTimeIfWindowSizeIsLargerThanDefaultRetentionTime() { + final long windowSize = 2 * Windows.DEFAULT_MAINTAIN_DURATION_MS; + assertEquals(windowSize, TimeWindows.of(windowSize).maintainMs()); + } @Test public void shouldHaveSaneEqualsAndHashCode() { - TimeWindows w1 = TimeWindows.of(anySize); - TimeWindows w2 = TimeWindows.of(w1.size); + TimeWindows w1 = TimeWindows.of(ANY_SIZE); + TimeWindows w2 = TimeWindows.of(w1.sizeMs); // Reflexive assertEquals(w1, w1); @@ -46,7 +69,7 @@ public void shouldHaveSaneEqualsAndHashCode() { assertEquals(w1.hashCode(), w2.hashCode()); // Transitive - TimeWindows w3 = TimeWindows.of(w2.size); + TimeWindows w3 = TimeWindows.of(w2.sizeMs); assertEquals(w2, w3); assertEquals(w1, w3); assertEquals(w1.hashCode(), w3.hashCode()); @@ -56,42 +79,69 @@ public void shouldHaveSaneEqualsAndHashCode() { assertNotEquals("must be false for different window types", UnlimitedWindows.of(), w1); assertNotEquals("must be false for different types", new Object(), w1); - TimeWindows differentWindowSize = TimeWindows.of(w1.size + 1); + TimeWindows differentWindowSize = TimeWindows.of(w1.sizeMs + 1); assertNotEquals("must be false when window sizes are different", differentWindowSize, w1); - TimeWindows differentAdvanceInterval = w1.advanceBy(w1.advance - 1); + TimeWindows differentAdvanceInterval = w1.advanceBy(w1.advanceMs - 1); assertNotEquals("must be false when advance intervals are different", differentAdvanceInterval, w1); } + @Test(expected = IllegalArgumentException.class) + public void windowSizeMustNotBeZero() { + TimeWindows.of(0); + } @Test(expected = IllegalArgumentException.class) public void windowSizeMustNotBeNegative() { TimeWindows.of(-1); } - @Test(expected = IllegalArgumentException.class) - public void windowSizeMustNotBeZero() { - TimeWindows.of(0); + @Test + public void advanceIntervalMustNotBeZero() { + final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE); + try { + windowSpec.advanceBy(0); + fail("should not accept zero advance parameter"); + } catch (final IllegalArgumentException e) { + // expected + } } - @Test(expected = IllegalArgumentException.class) + @Test public void advanceIntervalMustNotBeNegative() { - TimeWindows.of(anySize).advanceBy(-1); + final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE); + try { + windowSpec.advanceBy(-1); + fail("should not accept negative advance parameter"); + } catch (final IllegalArgumentException e) { + // expected + } } - @Test(expected = IllegalArgumentException.class) - public void advanceIntervalMustNotBeZero() { - TimeWindows.of(anySize).advanceBy(0); + @Test + public void advanceIntervalMustNotBeLargerThanWindowSize() { + final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE); + try { + windowSpec.advanceBy(ANY_SIZE + 1); + fail("should not accept advance greater than window size"); + } catch (final IllegalArgumentException e) { + // expected + } } - @Test(expected = IllegalArgumentException.class) - public void advanceIntervalMustNotBeLargerThanWindowSize() { - long size = anySize; - TimeWindows.of(size).advanceBy(size + 1); + @Test + public void retentionTimeMustNoBeSmallerThanWindowSize() { + final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE); + try { + windowSpec.until(ANY_SIZE - 1); + fail("should not accept retention time smaller than window size"); + } catch (final IllegalArgumentException e) { + // expected + } } @Test - public void windowsForHoppingWindows() { + public void shouldComputeWindowsForHoppingWindows() { TimeWindows windows = TimeWindows.of(12L).advanceBy(5L); Map matched = windows.windowsFor(21L); assertEquals(12L / 5L + 1, matched.size()); @@ -101,7 +151,7 @@ public void windowsForHoppingWindows() { } @Test - public void windowsForBarelyOverlappingHoppingWindows() { + public void shouldComputeWindowsForBarelyOverlappingHoppingWindows() { TimeWindows windows = TimeWindows.of(6L).advanceBy(5L); Map matched = windows.windowsFor(7L); assertEquals(1, matched.size()); @@ -109,7 +159,7 @@ public void windowsForBarelyOverlappingHoppingWindows() { } @Test - public void windowsForTumblingWindows() { + public void shouldComputeWindowsForTumblingWindows() { TimeWindows windows = TimeWindows.of(12L); Map matched = windows.windowsFor(21L); assertEquals(1, matched.size()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java index c1f4be6246353..ea9078c9dbaca 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java @@ -26,25 +26,37 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; public class UnlimitedWindowsTest { private static long anyStartTime = 10L; + @Test + public void shouldSetWindowStartTime() { + assertEquals(anyStartTime, UnlimitedWindows.of().startOn(anyStartTime).startMs); + } + @Test(expected = IllegalArgumentException.class) public void startTimeMustNotBeNegative() { UnlimitedWindows.of().startOn(-1); } @Test - public void startTimeCanBeZero() { - UnlimitedWindows.of().startOn(0); + public void shouldThrowOnUntil() { + final UnlimitedWindows windowSpec = UnlimitedWindows.of(); + try { + windowSpec.until(42); + fail("should not allow to set window retention time"); + } catch (final IllegalArgumentException e) { + // expected + } } @Test public void shouldIncludeRecordsThatHappenedOnWindowStart() { UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime); - Map matchedWindows = w.windowsFor(w.start); + Map matchedWindows = w.windowsFor(w.startMs); assertEquals(1, matchedWindows.size()); assertEquals(new UnlimitedWindow(anyStartTime), matchedWindows.get(anyStartTime)); } @@ -52,7 +64,7 @@ public void shouldIncludeRecordsThatHappenedOnWindowStart() { @Test public void shouldIncludeRecordsThatHappenedAfterWindowStart() { UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime); - long timestamp = w.start + 1; + long timestamp = w.startMs + 1; Map matchedWindows = w.windowsFor(timestamp); assertEquals(1, matchedWindows.size()); assertEquals(new UnlimitedWindow(anyStartTime), matchedWindows.get(anyStartTime)); @@ -61,7 +73,7 @@ public void shouldIncludeRecordsThatHappenedAfterWindowStart() { @Test public void shouldExcludeRecordsThatHappenedBeforeWindowStart() { UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime); - long timestamp = w.start - 1; + long timestamp = w.startMs - 1; Map matchedWindows = w.windowsFor(timestamp); assertTrue(matchedWindows.isEmpty()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/WindowTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowTest.java new file mode 100644 index 0000000000000..55c5c60d430a9 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowTest.java @@ -0,0 +1,85 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.kafka.streams.kstream; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +public class WindowTest { + + static class TestWindow extends Window { + TestWindow(final long startMs, final long endMs) { + super(startMs, endMs); + } + + @Override + public boolean overlap(final Window other) { + return false; + } + } + + static class TestWindow2 extends Window { + TestWindow2(final long startMs, final long endMs) { + super(startMs, endMs); + } + + @Override + public boolean overlap(final Window other) { + return false; + } + } + + private final TestWindow window = new TestWindow(5, 10); + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIfStartIsNegative() { + new TestWindow(-1, 0); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIfEndIsSmallerThanStart() { + new TestWindow(1, 0); + } + + @Test + public void shouldBeEqualIfStartAndEndSame() { + final TestWindow window2 = new TestWindow(window.startMs, window.endMs); + + assertEquals(window, window); + assertEquals(window, window2); + assertEquals(window2, window); + } + + @Test + public void shouldNotBeEqualIfStartOrEndIsDifferent() { + assertNotEquals(window, new TestWindow(0, window.endMs)); + assertNotEquals(window, new TestWindow(7, window.endMs)); + assertNotEquals(window, new TestWindow(window.startMs, 7)); + assertNotEquals(window, new TestWindow(window.startMs, 15)); + assertNotEquals(window, new TestWindow(7, 8)); + assertNotEquals(window, new TestWindow(0, 15)); + } + + @Test + public void shouldNotBeEqualIfDifferentWindowType() { + assertNotEquals(window, new TestWindow2(window.startMs, window.endMs)); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java new file mode 100644 index 0000000000000..890265f917776 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.kstream; + +import org.junit.Test; + +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class WindowsTest { + + private class TestWindows extends Windows { + + @Override + public Map windowsFor(long timestamp) { + return null; + } + + @Override + public long size() { + return 0; + } + } + + @Test + public void shouldSetNumberOfSegments() { + final int anySegmentSizeLargerThanOne = 5; + assertEquals(anySegmentSizeLargerThanOne, new TestWindows().segments(anySegmentSizeLargerThanOne).segments); + } + + @Test + public void shouldSetWindowRetentionTime() { + final int anyNotNegativeRetentionTime = 42; + assertEquals(anyNotNegativeRetentionTime, new TestWindows().until(anyNotNegativeRetentionTime).maintainMs()); + } + + @Test(expected = IllegalArgumentException.class) + public void numberOfSegmentsMustBeAtLeastTwo() { + new TestWindows().segments(1); + } + + @Test(expected = IllegalArgumentException.class) + public void retentionTimeMustNotBeNegative() { + new TestWindows().until(-1); + } + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java new file mode 100644 index 0000000000000..efa20b8be7464 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java @@ -0,0 +1,122 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.kafka.streams.kstream.internals; + +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class TimeWindowTest { + + private long start = 50; + private long end = 100; + private final TimeWindow window = new TimeWindow(start, end); + private final SessionWindow sessionWindow = new SessionWindow(start, end); + + @Test(expected = IllegalArgumentException.class) + public void endMustBeLargerThanStart() { + new TimeWindow(start, start); + } + + @Test + public void shouldNotOverlapIfOtherWindowIsBeforeThisWindow() { + /* + * This: [-------) + * Other: [-----) + */ + assertFalse(window.overlap(new TimeWindow(0, 25))); + assertFalse(window.overlap(new TimeWindow(0, start - 1))); + assertFalse(window.overlap(new TimeWindow(0, start))); + } + + @Test + public void shouldOverlapIfOtherWindowEndIsWithinThisWindow() { + /* + * This: [-------) + * Other: [---------) + */ + assertTrue(window.overlap(new TimeWindow(0, start + 1))); + assertTrue(window.overlap(new TimeWindow(0, 75))); + assertTrue(window.overlap(new TimeWindow(0, end - 1))); + + assertTrue(window.overlap(new TimeWindow(start - 1, start + 1))); + assertTrue(window.overlap(new TimeWindow(start - 1, 75))); + assertTrue(window.overlap(new TimeWindow(start - 1, end - 1))); + } + + @Test + public void shouldOverlapIfOtherWindowContainsThisWindow() { + /* + * This: [-------) + * Other: [------------------) + */ + assertTrue(window.overlap(new TimeWindow(0, end))); + assertTrue(window.overlap(new TimeWindow(0, end + 1))); + assertTrue(window.overlap(new TimeWindow(0, 150))); + + assertTrue(window.overlap(new TimeWindow(start - 1, end))); + assertTrue(window.overlap(new TimeWindow(start - 1, end + 1))); + assertTrue(window.overlap(new TimeWindow(start - 1, 150))); + + assertTrue(window.overlap(new TimeWindow(start, end))); + assertTrue(window.overlap(new TimeWindow(start, end + 1))); + assertTrue(window.overlap(new TimeWindow(start, 150))); + } + + @Test + public void shouldOverlapIfOtherWindowIsWithinThisWindow() { + /* + * This: [-------) + * Other: [---) + */ + assertTrue(window.overlap(new TimeWindow(start, 75))); + assertTrue(window.overlap(new TimeWindow(start, end))); + assertTrue(window.overlap(new TimeWindow(75, end))); + } + + @Test + public void shouldOverlapIfOtherWindowStartIsWithinThisWindow() { + /* + * This: [-------) + * Other: [-------) + */ + assertTrue(window.overlap(new TimeWindow(start, end + 1))); + assertTrue(window.overlap(new TimeWindow(start, 150))); + assertTrue(window.overlap(new TimeWindow(75, end + 1))); + assertTrue(window.overlap(new TimeWindow(75, 150))); + } + + @Test + public void shouldNotOverlapIsOtherWindowIsAfterThisWindow() { + /* + * This: [-------) + * Other: [------) + */ + assertFalse(window.overlap(new TimeWindow(end, end + 1))); + assertFalse(window.overlap(new TimeWindow(end, 150))); + assertFalse(window.overlap(new TimeWindow(end + 1, 150))); + assertFalse(window.overlap(new TimeWindow(125, 150))); + } + + @Test(expected = IllegalArgumentException.class) + public void cannotCompareTimeWindowWithDifferentWindowType() { + window.overlap(sessionWindow); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindowTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindowTest.java new file mode 100644 index 0000000000000..f3c9cfbff89c9 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindowTest.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.kafka.streams.kstream.internals; + +import org.junit.Test; + +import static org.junit.Assert.assertTrue; + +public class UnlimitedWindowTest { + + private long start = 50; + private final UnlimitedWindow window = new UnlimitedWindow(start); + private final SessionWindow sessionWindow = new SessionWindow(start, start); + + @Test + public void shouldAlwaysOverlap() { + assertTrue(window.overlap(new UnlimitedWindow(start - 1))); + assertTrue(window.overlap(new UnlimitedWindow(start))); + assertTrue(window.overlap(new UnlimitedWindow(start + 1))); + } + + @Test(expected = IllegalArgumentException.class) + public void cannotCompareUnlimitedWindowWithDifferentWindowType() { + window.overlap(sessionWindow); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java index 4be83bebf2566..0b6288fb4d4dd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java @@ -81,7 +81,7 @@ public void testCopartitioning() { Integer expected = defaultPartitioner.partition("topic", key, keyBytes, value, valueBytes, cluster); - for (int w = 0; w < 10; w++) { + for (int w = 1; w < 10; w++) { TimeWindow window = new TimeWindow(10 * w, 20 * w); Windowed windowedKey = new Windowed<>(key, window); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java index 8bfcb7bfb49d5..51b3bf0a4b5e6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java @@ -20,7 +20,7 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.kstream.internals.TimeWindow; +import org.apache.kafka.streams.kstream.internals.SessionWindow; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.QueryableStoreTypes; import org.apache.kafka.test.ReadOnlySessionStoreStub; @@ -58,12 +58,12 @@ public void before() { @Test public void shouldFetchResulstFromUnderlyingSessionStore() throws Exception { - underlyingSessionStore.put(new Windowed<>("a", new TimeWindow(0, 0)), 1L); - underlyingSessionStore.put(new Windowed<>("a", new TimeWindow(10, 10)), 2L); + underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 1L); + underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(10, 10)), 2L); final List, Long>> results = toList(sessionStore.fetch("a")); - assertEquals(Arrays.asList(KeyValue.pair(new Windowed<>("a", new TimeWindow(0, 0)), 1L), - KeyValue.pair(new Windowed<>("a", new TimeWindow(10, 10)), 2L)), + assertEquals(Arrays.asList(KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 1L), + KeyValue.pair(new Windowed<>("a", new SessionWindow(10, 10)), 2L)), results); } @@ -79,8 +79,8 @@ public void shouldFindValueForKeyWhenMultiStores() throws Exception { ReadOnlySessionStoreStub<>(); stubProviderTwo.addStore(storeName, secondUnderlying); - final Windowed keyOne = new Windowed<>("key-one", new TimeWindow(0, 0)); - final Windowed keyTwo = new Windowed<>("key-two", new TimeWindow(0, 0)); + final Windowed keyOne = new Windowed<>("key-one", new SessionWindow(0, 0)); + final Windowed keyTwo = new Windowed<>("key-two", new SessionWindow(0, 0)); underlyingSessionStore.put(keyOne, 0L); secondUnderlying.put(keyTwo, 10L); @@ -93,8 +93,8 @@ public void shouldFindValueForKeyWhenMultiStores() throws Exception { @Test public void shouldNotGetValueFromOtherStores() throws Exception { - final Windowed expectedKey = new Windowed<>("foo", new TimeWindow(0, 0)); - otherUnderlyingStore.put(new Windowed<>("foo", new TimeWindow(10, 10)), 10L); + final Windowed expectedKey = new Windowed<>("foo", new SessionWindow(0, 0)); + otherUnderlyingStore.put(new Windowed<>("foo", new SessionWindow(10, 10)), 10L); underlyingSessionStore.put(expectedKey, 1L); final KeyValueIterator, Long> result = sessionStore.fetch("foo"); From 928f69ce883c8e30050d2e6f5e72e77e6740bdb9 Mon Sep 17 00:00:00 2001 From: huxi Date: Tue, 24 Jan 2017 10:04:42 +0000 Subject: [PATCH 029/177] KAFKA-4576; Log segments close to max size break on fetch `FileChannel.read` may not fill the destination buffer even if there are enough bytes in the channel to do so. Add a couple of utility methods that ensure this and use them from all the relevant places. Author: huxi Author: amethystic Author: Ismael Juma Reviewers: Jun Rao , Jason Gustafson , Ismael Juma Closes #2304 from amethystic/kafka4576_FileChannel_read (cherry picked from commit 337f576f5979bf8924c5707b338cf4d3c76a53fd) Signed-off-by: Ismael Juma --- .../common/record/FileLogInputStream.java | 13 +- .../kafka/common/record/FileRecords.java | 9 +- .../org/apache/kafka/common/utils/Utils.java | 52 +++++++ .../apache/kafka/common/utils/UtilsTest.java | 140 +++++++++++++++++- 4 files changed, 201 insertions(+), 13 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java index ae393b03c0d62..dd9cc84dfb253 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.errors.CorruptRecordException; +import org.apache.kafka.common.utils.Utils; import java.io.IOException; import java.nio.ByteBuffer; @@ -56,9 +57,7 @@ public FileChannelLogEntry nextEntry() throws IOException { return null; logHeaderBuffer.rewind(); - channel.read(logHeaderBuffer, position); - if (logHeaderBuffer.hasRemaining()) - return null; + Utils.readFullyOrFail(channel, logHeaderBuffer, position, "log header"); logHeaderBuffer.rewind(); long offset = logHeaderBuffer.getLong(); @@ -117,9 +116,7 @@ public byte magic() { try { byte[] magic = new byte[1]; ByteBuffer buf = ByteBuffer.wrap(magic); - channel.read(buf, position + Records.LOG_OVERHEAD + Record.MAGIC_OFFSET); - if (buf.hasRemaining()) - throw new KafkaException("Failed to read magic byte from FileChannel " + channel); + Utils.readFullyOrFail(channel, buf, position + Records.LOG_OVERHEAD + Record.MAGIC_OFFSET, "magic byte"); return magic[0]; } catch (IOException e) { throw new KafkaException(e); @@ -136,9 +133,7 @@ private Record loadRecord() throws IOException { return record; ByteBuffer recordBuffer = ByteBuffer.allocate(recordSize); - channel.read(recordBuffer, position + Records.LOG_OVERHEAD); - if (recordBuffer.hasRemaining()) - throw new IOException("Failed to read full record from channel " + channel); + Utils.readFullyOrFail(channel, recordBuffer, position + Records.LOG_OVERHEAD, "full record"); recordBuffer.rewind(); record = new Record(recordBuffer); diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java index 8a33dcaaef828..960b7168ff70f 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java @@ -108,14 +108,17 @@ public FileChannel channel() { } /** - * Read log entries into a given buffer. + * Read log entries into the given buffer until there are no bytes remaining in the buffer or the end of the file + * is reached. + * * @param buffer The buffer to write the entries to * @param position Position in the buffer to read from * @return The same buffer - * @throws IOException + * @throws IOException If an I/O error occurs, see {@link FileChannel#read(ByteBuffer, long)} for details on the + * possible exceptions */ public ByteBuffer readInto(ByteBuffer buffer, int position) throws IOException { - channel.read(buffer, position + this.start); + Utils.readFully(channel, buffer, position + this.start); buffer.flip(); return buffer; } diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index ac8d0786d7bc4..afa85bd23e9b2 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -12,6 +12,7 @@ */ package org.apache.kafka.common.utils; +import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.io.Closeable; @@ -803,4 +804,55 @@ public static long computeChecksum(ByteBuffer buffer, int start, int size) { return Crc32.crc32(buffer.array(), buffer.arrayOffset() + start, size); } + /** + * Read data from the channel to the given byte buffer until there are no bytes remaining in the buffer. If the end + * of the file is reached while there are bytes remaining in the buffer, an EOFException is thrown. + * + * @param channel File channel containing the data to read from + * @param destinationBuffer The buffer into which bytes are to be transferred + * @param position The file position at which the transfer is to begin; it must be non-negative + * @param description A description of what is being read, this will be included in the EOFException if it is thrown + * + * @throws IllegalArgumentException If position is negative + * @throws EOFException If the end of the file is reached while there are remaining bytes in the destination buffer + * @throws IOException If an I/O error occurs, see {@link FileChannel#read(ByteBuffer, long)} for details on the + * possible exceptions + */ + public static void readFullyOrFail(FileChannel channel, ByteBuffer destinationBuffer, long position, + String description) throws IOException { + if (position < 0) { + throw new IllegalArgumentException("The file channel position cannot be negative, but it is " + position); + } + int expectedReadBytes = destinationBuffer.remaining(); + readFully(channel, destinationBuffer, position); + if (destinationBuffer.hasRemaining()) { + throw new EOFException(String.format("Failed to read `%s` from file channel `%s`. Expected to read %d bytes, " + + "but reached end of file after reading %d bytes. Started read from position %d.", + description, channel, expectedReadBytes, expectedReadBytes - destinationBuffer.remaining(), position)); + } + } + + /** + * Read data from the channel to the given byte buffer until there are no bytes remaining in the buffer or the end + * of the file has been reached. + * + * @param channel File channel containing the data to read from + * @param destinationBuffer The buffer into which bytes are to be transferred + * @param position The file position at which the transfer is to begin; it must be non-negative + * + * @throws IllegalArgumentException If position is negative + * @throws IOException If an I/O error occurs, see {@link FileChannel#read(ByteBuffer, long)} for details on the + * possible exceptions + */ + public static void readFully(FileChannel channel, ByteBuffer destinationBuffer, long position) throws IOException { + if (position < 0) { + throw new IllegalArgumentException("The file channel position cannot be negative, but it is " + position); + } + long currentPosition = position; + int bytesRead; + do { + bytesRead = channel.read(destinationBuffer, currentPosition); + currentPosition += bytesRead; + } while (bytesRead != -1 && destinationBuffer.hasRemaining()); + } } diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java index 46400b4a63474..194cad6807422 100755 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -16,18 +16,27 @@ */ package org.apache.kafka.common.utils; +import java.io.EOFException; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; import java.util.Arrays; import java.util.Collections; import java.io.Closeable; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Random; +import org.apache.kafka.test.TestUtils; +import org.easymock.EasyMock; +import org.easymock.IAnswer; import org.junit.Test; + +import static org.apache.kafka.common.utils.Utils.formatAddress; import static org.apache.kafka.common.utils.Utils.getHost; import static org.apache.kafka.common.utils.Utils.getPort; -import static org.apache.kafka.common.utils.Utils.formatAddress; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -158,6 +167,135 @@ public void testCloseAll() { } } + @Test + public void testReadFullyOrFailWithRealFile() throws IOException { + try (FileChannel channel = FileChannel.open(TestUtils.tempFile().toPath(), StandardOpenOption.READ, StandardOpenOption.WRITE)) { + // prepare channel + String msg = "hello, world"; + channel.write(ByteBuffer.wrap(msg.getBytes()), 0); + channel.force(true); + assertEquals("Message should be written to the file channel", channel.size(), msg.length()); + + ByteBuffer perfectBuffer = ByteBuffer.allocate(msg.length()); + ByteBuffer smallBuffer = ByteBuffer.allocate(5); + ByteBuffer largeBuffer = ByteBuffer.allocate(msg.length() + 1); + // Scenario 1: test reading into a perfectly-sized buffer + Utils.readFullyOrFail(channel, perfectBuffer, 0, "perfect"); + assertFalse("Buffer should be filled up", perfectBuffer.hasRemaining()); + assertEquals("Buffer should be populated correctly", msg, new String(perfectBuffer.array())); + // Scenario 2: test reading into a smaller buffer + Utils.readFullyOrFail(channel, smallBuffer, 0, "small"); + assertFalse("Buffer should be filled", smallBuffer.hasRemaining()); + assertEquals("Buffer should be populated correctly", "hello", new String(smallBuffer.array())); + // Scenario 3: test reading starting from a non-zero position + smallBuffer.clear(); + Utils.readFullyOrFail(channel, smallBuffer, 7, "small"); + assertFalse("Buffer should be filled", smallBuffer.hasRemaining()); + assertEquals("Buffer should be populated correctly", "world", new String(smallBuffer.array())); + // Scenario 4: test end of stream is reached before buffer is filled up + try { + Utils.readFullyOrFail(channel, largeBuffer, 0, "large"); + fail("Expected EOFException to be raised"); + } catch (EOFException e) { + // expected + } + } + } + + /** + * Tests that `readFullyOrFail` behaves correctly if multiple `FileChannel.read` operations are required to fill + * the destination buffer. + */ + @Test + public void testReadFullyOrFailWithPartialFileChannelReads() throws IOException { + FileChannel channelMock = EasyMock.createMock(FileChannel.class); + final int bufferSize = 100; + ByteBuffer buffer = ByteBuffer.allocate(bufferSize); + StringBuilder expectedBufferContent = new StringBuilder(); + fileChannelMockExpectReadWithRandomBytes(channelMock, expectedBufferContent, bufferSize); + EasyMock.replay(channelMock); + Utils.readFullyOrFail(channelMock, buffer, 0L, "test"); + assertEquals("The buffer should be populated correctly", expectedBufferContent.toString(), + new String(buffer.array())); + assertFalse("The buffer should be filled", buffer.hasRemaining()); + EasyMock.verify(channelMock); + } + + /** + * Tests that `readFullyOrFail` behaves correctly if multiple `FileChannel.read` operations are required to fill + * the destination buffer. + */ + @Test + public void testReadFullyWithPartialFileChannelReads() throws IOException { + FileChannel channelMock = EasyMock.createMock(FileChannel.class); + final int bufferSize = 100; + StringBuilder expectedBufferContent = new StringBuilder(); + fileChannelMockExpectReadWithRandomBytes(channelMock, expectedBufferContent, bufferSize); + EasyMock.replay(channelMock); + ByteBuffer buffer = ByteBuffer.allocate(bufferSize); + Utils.readFully(channelMock, buffer, 0L); + assertEquals("The buffer should be populated correctly.", expectedBufferContent.toString(), + new String(buffer.array())); + assertFalse("The buffer should be filled", buffer.hasRemaining()); + EasyMock.verify(channelMock); + } + + @Test + public void testReadFullyIfEofIsReached() throws IOException { + final FileChannel channelMock = EasyMock.createMock(FileChannel.class); + final int bufferSize = 100; + final String fileChannelContent = "abcdefghkl"; + ByteBuffer buffer = ByteBuffer.allocate(bufferSize); + EasyMock.expect(channelMock.size()).andReturn((long) fileChannelContent.length()); + EasyMock.expect(channelMock.read(EasyMock.anyObject(ByteBuffer.class), EasyMock.anyInt())).andAnswer(new IAnswer() { + @Override + public Integer answer() throws Throwable { + ByteBuffer buffer = (ByteBuffer) EasyMock.getCurrentArguments()[0]; + buffer.put(fileChannelContent.getBytes()); + return -1; + } + }); + EasyMock.replay(channelMock); + Utils.readFully(channelMock, buffer, 0L); + assertEquals("abcdefghkl", new String(buffer.array(), 0, buffer.position())); + assertEquals(buffer.position(), channelMock.size()); + assertTrue(buffer.hasRemaining()); + EasyMock.verify(channelMock); + } + + /** + * Expectation setter for multiple reads where each one reads random bytes to the buffer. + * + * @param channelMock The mocked FileChannel object + * @param expectedBufferContent buffer that will be updated to contain the expected buffer content after each + * `FileChannel.read` invocation + * @param bufferSize The buffer size + * @throws IOException If an I/O error occurs + */ + private void fileChannelMockExpectReadWithRandomBytes(final FileChannel channelMock, + final StringBuilder expectedBufferContent, + final int bufferSize) throws IOException { + final int step = 20; + final Random random = new Random(); + int remainingBytes = bufferSize; + while (remainingBytes > 0) { + final int mockedBytesRead = remainingBytes < step ? remainingBytes : random.nextInt(step); + final StringBuilder sb = new StringBuilder(); + EasyMock.expect(channelMock.read(EasyMock.anyObject(ByteBuffer.class), EasyMock.anyInt())).andAnswer(new IAnswer() { + @Override + public Integer answer() throws Throwable { + ByteBuffer buffer = (ByteBuffer) EasyMock.getCurrentArguments()[0]; + for (int i = 0; i < mockedBytesRead; i++) + sb.append("a"); + buffer.put(sb.toString().getBytes()); + expectedBufferContent.append(sb); + return mockedBytesRead; + } + }); + remainingBytes -= mockedBytesRead; + } + } + private static class TestCloseable implements Closeable { private final int id; private final IOException closeException; From b676f51dd54125b950fee35fe867057778866bcb Mon Sep 17 00:00:00 2001 From: "Colin P. Mccabe" Date: Tue, 24 Jan 2017 11:09:47 +0000 Subject: [PATCH 030/177] KAFKA-4688; Kafka 0.10.1.1 should be available in system tests Author: Colin P. Mccabe Reviewers: Ewen Cheslack-Postava , Ismael Juma Closes #2424 from cmccabe/KAFKA-4688 (cherry picked from commit 567180605c0a8cf74b8d2cad52e2873c83065a95) Signed-off-by: Ismael Juma --- tests/docker/Dockerfile | 1 + tests/kafkatest/version.py | 3 ++- vagrant/base.sh | 2 ++ 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/docker/Dockerfile b/tests/docker/Dockerfile index bdd6af612dd11..82e2e55f4320a 100644 --- a/tests/docker/Dockerfile +++ b/tests/docker/Dockerfile @@ -28,6 +28,7 @@ ENV MIRROR="http://apache.cs.utah.edu/" RUN wget -q "${MIRROR}kafka/0.8.2.2/kafka_2.10-0.8.2.2.tgz" -O "/tmp/kafka_2.10-0.8.2.2.tgz" && tar xfz /tmp/kafka_2.10-0.8.2.2.tgz -C /opt && mv "/opt/kafka_2.10-0.8.2.2" "/opt/kafka-0.8.2.2" RUN wget -q "${MIRROR}kafka/0.9.0.1/kafka_2.10-0.9.0.1.tgz" -O "/tmp/kafka_2.10-0.9.0.1.tgz" && tar xfz /tmp/kafka_2.10-0.9.0.1.tgz -C /opt && mv "/opt/kafka_2.10-0.9.0.1" "/opt/kafka-0.9.0.1" RUN wget -q "${MIRROR}kafka/0.10.0.1/kafka_2.10-0.10.0.1.tgz" -O "/tmp/kafka_2.10-0.10.0.1.tgz" && tar xfz /tmp/kafka_2.10-0.10.0.1.tgz -C /opt && mv "/opt/kafka_2.10-0.10.0.1" "/opt/kafka-0.10.0.1" +RUN wget -q "${MIRROR}kafka/0.10.1.1/kafka_2.10-0.10.1.1.tgz" -O "/tmp/kafka_2.10-0.10.1.1.tgz" && tar xfz /tmp/kafka_2.10-0.10.1.1.tgz -C /opt && mv "/opt/kafka_2.10-0.10.1.1" "/opt/kafka-0.10.1.1" RUN rm /tmp/kafka_*.tgz ADD ssh /root/.ssh diff --git a/tests/kafkatest/version.py b/tests/kafkatest/version.py index 5dd09a1fa5150..cd361e836b49d 100644 --- a/tests/kafkatest/version.py +++ b/tests/kafkatest/version.py @@ -82,6 +82,7 @@ def get_version(node=None): # 0.10.1.x versions V_0_10_1_0 = KafkaVersion("0.10.1.0") -LATEST_0_10_1 = V_0_10_1_0 +V_0_10_1_1 = KafkaVersion("0.10.1.1") +LATEST_0_10_1 = V_0_10_1_1 LATEST_0_10 = LATEST_0_10_1 diff --git a/vagrant/base.sh b/vagrant/base.sh index 369776584a2fd..5f07cc3658b36 100755 --- a/vagrant/base.sh +++ b/vagrant/base.sh @@ -67,6 +67,8 @@ get_kafka 0.9.0.1 chmod a+rw /opt/kafka-0.9.0.1 get_kafka 0.10.0.1 chmod a+rw /opt/kafka-0.10.0.1 +get_kafka 0.10.1.1 +chmod a+rw /opt/kafka-0.10.1.1 # For EC2 nodes, we want to use /mnt, which should have the local disk. On local From 90b9a8a3cc7f9443101f9f95c0acc87873453f68 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Tue, 24 Jan 2017 12:41:21 +0000 Subject: [PATCH 031/177] MINOR: Add expected Error Codes to ProduceResponse documentation This is a documentation-only patch discussed on the mailing list. The intent is to have these changes propagated to the protocol wiki (https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol) . Author: Dana Powers Reviewers: Grant Henke , Ismael Juma Closes #918 from dpkp/produce_response_errors (cherry picked from commit 3bb0d3cd888c29a002400253d9ff84ce73ebc03c) Signed-off-by: Ismael Juma --- .../apache/kafka/common/requests/ProduceResponse.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 71e6ab5cfde24..6c2125a98db05 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -48,7 +48,16 @@ public class ProduceResponse extends AbstractResponse { /** * Possible error code: * - * TODO + * CORRUPT_MESSAGE (2) + * UNKNOWN_TOPIC_OR_PARTITION (3) + * NOT_LEADER_FOR_PARTITION (6) + * MESSAGE_TOO_LARGE (10) + * INVALID_TOPIC (17) + * RECORD_LIST_TOO_LARGE (18) + * NOT_ENOUGH_REPLICAS (19) + * NOT_ENOUGH_REPLICAS_AFTER_APPEND (20) + * INVALID_REQUIRED_ACKS (21) + * TOPIC_AUTHORIZATION_FAILED (29) */ private static final String BASE_OFFSET_KEY_NAME = "base_offset"; From 740abdb7bf6dff841a09baf84f1bacd88401f50e Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Tue, 24 Jan 2017 11:02:34 -0800 Subject: [PATCH 032/177] MINOR: rework JavaDoc for windowing related public API - also some code refactoring and bug fixes Author: Matthias J. Sax Reviewers: Damian Guy, Eno Thereska, Guozhang Wang Closes #2337 from mjsax/javaDocImprovements4 (cherry picked from commit b938c03b0aa7a0cebdaad0cbb22689cc719c6ebc) Signed-off-by: Guozhang Wang --- .../kafka/streams/kstream/JoinWindows.java | 90 +++++++++++---- .../kafka/streams/kstream/SessionWindows.java | 60 ++++++---- .../kafka/streams/kstream/TimeWindows.java | 107 +++++++++++------- .../streams/kstream/UnlimitedWindows.java | 83 ++++++++++---- .../apache/kafka/streams/kstream/Window.java | 40 ++++--- .../kafka/streams/kstream/Windowed.java | 28 ++++- .../apache/kafka/streams/kstream/Windows.java | 61 ++++++---- .../kstream/internals/SessionWindow.java | 18 ++- .../streams/kstream/internals/TimeWindow.java | 37 +++++- .../kstream/internals/UnlimitedWindow.java | 34 +++++- 10 files changed, 403 insertions(+), 155 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java index 6dd1a85f5340a..f20e39f6a4d29 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java @@ -5,9 +5,9 @@ * 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 - * + *

      + * 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. @@ -17,20 +17,23 @@ package org.apache.kafka.streams.kstream; +import org.apache.kafka.common.annotation.InterfaceStability; + import java.util.Map; /** * The window specifications used for joins. *

      - * A {@link JoinWindows} instance defines a join over two stream on the same key and a maximum time difference. + * A {@link JoinWindows} instance defines a maximum time difference for a {@link KStream#join(KStream, ValueJoiner, + * JoinWindows) join over two streams} on the same key. * In SQL-style you would express this join as - *

      + * 
      {@code
        *     SELECT * FROM stream1, stream2
        *     WHERE
        *       stream1.key = stream2.key
        *       AND
        *       stream1.ts - before <= stream2.ts AND stream2.ts <= stream1.ts + after
      - * 
      + * }
      * There are three different window configuration supported: *