├── .github ├── CODEOWNERS └── workflows │ └── maven.yml ├── .gitignore ├── ADOPTERS.md ├── CODE_OF_CONDUCT.md ├── CONTRIBUTING.md ├── LICENSE ├── README.md ├── SECURITY.md ├── docs ├── autoremediation.md ├── images │ └── psc-architecture.jpg ├── nativekafkatopscmigration.md ├── quickstart.md ├── resourcenames.md └── servicediscovery.md ├── pom.xml ├── psc-common ├── README.md ├── pom.xml └── src │ ├── main │ └── java │ │ └── com │ │ └── pinterest │ │ └── psc │ │ └── common │ │ ├── ByteArrayHandler.java │ │ └── PscCommon.java │ └── test │ └── java │ └── com │ └── pinterest │ └── psc │ └── common │ └── TestPscCommon.java ├── psc-examples ├── README.md ├── pom.xml ├── quickstart-scripts │ ├── common.sh │ ├── configs │ │ ├── local-region.discovery.local-cluster.serverset │ │ └── local-region.discovery.local-cluster_tls.serverset │ ├── example_psc_consumer.sh │ ├── example_psc_producer.sh │ ├── startlocalkafka.sh │ └── stoplocalkafka.sh └── src │ └── main │ ├── java │ └── com │ │ └── pinterest │ │ └── psc │ │ ├── discovery │ │ └── ExampleKafkaServersetServiceDiscoveryProvider.java │ │ └── example │ │ ├── common │ │ ├── Commons.java │ │ ├── Message.java │ │ ├── MessageDeserializer.java │ │ ├── MessageSerializer.java │ │ └── TestMessageSerDe.java │ │ ├── kafka │ │ ├── ExamplePscConsumer.java │ │ └── ExamplePscProducer.java │ │ └── migration │ │ ├── consumer │ │ ├── Kafka.java │ │ └── Psc.java │ │ └── producer │ │ ├── Kafka.java │ │ └── Psc.java │ └── resources │ ├── log4j.xml │ ├── psc-override.conf │ ├── psc_kafka_consumer.conf │ └── psc_memq_consumer.conf ├── psc-flink-logging ├── README.md ├── pom.xml └── src │ └── main │ └── java │ └── com │ └── pinterest │ └── psc │ └── logging │ ├── PscLogger.java │ ├── PscLoggerManager.java │ ├── PscLoggerType.java │ └── slf4j │ └── PscSlf4jLogger.java ├── psc-flink ├── README.md ├── pom.xml └── src │ ├── main │ ├── java │ │ └── com │ │ │ └── pinterest │ │ │ └── flink │ │ │ ├── streaming │ │ │ ├── connectors │ │ │ │ └── psc │ │ │ │ │ ├── FlinkPscConsumer.java │ │ │ │ │ ├── FlinkPscConsumerBase.java │ │ │ │ │ ├── FlinkPscErrorCode.java │ │ │ │ │ ├── FlinkPscException.java │ │ │ │ │ ├── FlinkPscProducer.java │ │ │ │ │ ├── FlinkPscProducer011.java │ │ │ │ │ ├── FlinkPscProducerBase.java │ │ │ │ │ ├── PscContextAware.java │ │ │ │ │ ├── PscDeserializationSchema.java │ │ │ │ │ ├── PscSerializationSchema.java │ │ │ │ │ ├── PscTableSink.java │ │ │ │ │ ├── PscTableSinkBase.java │ │ │ │ │ ├── PscTableSource.java │ │ │ │ │ ├── PscTableSourceBase.java │ │ │ │ │ ├── PscTableSourceSinkFactory.java │ │ │ │ │ ├── PscTableSourceSinkFactoryBase.java │ │ │ │ │ ├── config │ │ │ │ │ ├── OffsetCommitMode.java │ │ │ │ │ ├── OffsetCommitModes.java │ │ │ │ │ └── StartupMode.java │ │ │ │ │ ├── internals │ │ │ │ │ ├── AbstractFetcher.java │ │ │ │ │ ├── AbstractTopicUriPartitionDiscoverer.java │ │ │ │ │ ├── ClosableBlockingQueue.java │ │ │ │ │ ├── ExceptionProxy.java │ │ │ │ │ ├── FlinkPscInternalProducer.java │ │ │ │ │ ├── Handover.java │ │ │ │ │ ├── KeyedSerializationSchemaWrapper.java │ │ │ │ │ ├── PscCommitCallback.java │ │ │ │ │ ├── PscConsumerThread.java │ │ │ │ │ ├── PscDeserializationSchemaWrapper.java │ │ │ │ │ ├── PscFetcher.java │ │ │ │ │ ├── PscSerializationSchemaWrapper.java │ │ │ │ │ ├── PscShuffleFetcher.java │ │ │ │ │ ├── PscSimpleTypeSerializerSnapshot.java │ │ │ │ │ ├── PscTopicUriPartition.java │ │ │ │ │ ├── PscTopicUriPartitionAssigner.java │ │ │ │ │ ├── PscTopicUriPartitionDiscoverer.java │ │ │ │ │ ├── PscTopicUriPartitionLeader.java │ │ │ │ │ ├── PscTopicUriPartitionState.java │ │ │ │ │ ├── PscTopicUriPartitionStateSentinel.java │ │ │ │ │ ├── PscTopicUriPartitionStateWithWatermarkGenerator.java │ │ │ │ │ ├── PscTopicUrisDescriptor.java │ │ │ │ │ ├── PscTopicUrisDescriptor.java~Stashed changes │ │ │ │ │ ├── SourceContextWatermarkOutputAdapter.java │ │ │ │ │ ├── TransactionalIdsGenerator.java │ │ │ │ │ └── metrics │ │ │ │ │ │ ├── FlinkPscStateRecoveryMetricConstants.java │ │ │ │ │ │ ├── PscConsumerMetricConstants.java │ │ │ │ │ │ ├── PscMetricMutableWrapper.java │ │ │ │ │ │ └── PscMetricWrapper.java │ │ │ │ │ ├── partitioner │ │ │ │ │ ├── FlinkFixedPartitioner.java │ │ │ │ │ └── FlinkPscPartitioner.java │ │ │ │ │ ├── shuffle │ │ │ │ │ ├── FlinkPscShuffle.java │ │ │ │ │ ├── FlinkPscShuffleConsumer.java │ │ │ │ │ ├── FlinkPscShuffleProducer.java │ │ │ │ │ └── StreamPscShuffleSink.java │ │ │ │ │ └── table │ │ │ │ │ ├── PscDynamicSink.java │ │ │ │ │ ├── PscDynamicSinkBase.java │ │ │ │ │ ├── PscDynamicSource.java │ │ │ │ │ ├── PscDynamicSourceBase.java │ │ │ │ │ ├── PscDynamicTableFactory.java │ │ │ │ │ ├── PscDynamicTableFactoryBase.java │ │ │ │ │ └── PscOptions.java │ │ │ └── util │ │ │ │ └── serialization │ │ │ │ └── psc │ │ │ │ ├── JSONKeyValueDeserializationSchema.java │ │ │ │ ├── KeyedDeserializationSchema.java │ │ │ │ ├── KeyedSerializationSchema.java │ │ │ │ └── TypeInformationKeyValueSerializationSchema.java │ │ │ └── table │ │ │ └── descriptors │ │ │ └── psc │ │ │ ├── Psc.java │ │ │ └── PscValidator.java │ └── resources │ │ └── META-INF │ │ └── services │ │ ├── org.apache.flink.table.factories.Factory │ │ └── org.apache.flink.table.factories.TableFactory │ └── test │ ├── java │ └── com │ │ └── pinterest │ │ ├── flink │ │ ├── streaming │ │ │ └── connectors │ │ │ │ └── psc │ │ │ │ ├── BasicPscConsumerTest.java │ │ │ │ ├── FlinkFixedPartitionerTest.java │ │ │ │ ├── FlinkPscConsumerBaseMigrationTest.java │ │ │ │ ├── FlinkPscConsumerBaseTest.java │ │ │ │ ├── FlinkPscInternalProducerITCase.java │ │ │ │ ├── FlinkPscProducerBaseTest.java │ │ │ │ ├── FlinkPscProducerITCase.java │ │ │ │ ├── FlinkPscProducerMigrationTest.java │ │ │ │ ├── FlinkPscProducerStateSerializerTest.java │ │ │ │ ├── FlinkPscProducerTest.java │ │ │ │ ├── JSONKeyValueDeserializationSchemaTest.java │ │ │ │ ├── NextTransactionalIdHintSerializerTest.java │ │ │ │ ├── PscConsumerTestBaseWithKafkaAsPubSub.java │ │ │ │ ├── PscITCase.java │ │ │ │ ├── PscMigrationTestBase.java │ │ │ │ ├── PscProducerAtLeastOnceITCase.java │ │ │ │ ├── PscProducerExactlyOnceITCase.java │ │ │ │ ├── PscProducerTestBase.java │ │ │ │ ├── PscSerializerUpgradeTest.java │ │ │ │ ├── PscShortRetentionTestBase.java │ │ │ │ ├── PscTableSourceSinkFactoryTest.java │ │ │ │ ├── PscTableSourceSinkFactoryTestBase.java │ │ │ │ ├── PscTestBaseWithFlinkWithKafkaAsPubSub.java │ │ │ │ ├── PscTestBaseWithKafkaAsPubSub.java │ │ │ │ ├── PscTestEnvironmentWithKafkaAsPubSub.java │ │ │ │ ├── PscTestEnvironmentWithKafkaAsPubSubImpl.java │ │ │ │ ├── internals │ │ │ │ ├── AbstractFetcherTest.java │ │ │ │ ├── AbstractFetcherWatermarksTest.java │ │ │ │ ├── AbstractPartitionDiscovererTest.java │ │ │ │ ├── ClosableBlockingQueueTest.java │ │ │ │ ├── PscTopicUriPartitionTest.java │ │ │ │ └── PscTopicUrisDescriptorTest.java │ │ │ │ ├── shuffle │ │ │ │ ├── PscShuffleExactlyOnceITCase.java │ │ │ │ ├── PscShuffleITCase.java │ │ │ │ └── PscShuffleTestBase.java │ │ │ │ ├── table │ │ │ │ ├── PscChangelogTableITCase.java │ │ │ │ ├── PscDynamicTableFactoryTest.java │ │ │ │ ├── PscDynamicTableFactoryTestBase.java │ │ │ │ ├── PscTableITCase.java │ │ │ │ └── PscTableTestBase.java │ │ │ │ └── testutils │ │ │ │ ├── ClusterCommunicationUtils.java │ │ │ │ ├── DataGenerators.java │ │ │ │ ├── FailingIdentityMapper.java │ │ │ │ ├── FakeStandardPscProducerConfig.java │ │ │ │ ├── IntegerSource.java │ │ │ │ ├── PartitionValidatingMapper.java │ │ │ │ ├── TestPartitionDiscoverer.java │ │ │ │ ├── TestSourceContext.java │ │ │ │ ├── TestTopicUriPartitionDiscoverer.java │ │ │ │ ├── ThrottledMapper.java │ │ │ │ ├── Tuple2FlinkPartitioner.java │ │ │ │ ├── ValidatingExactlyOnceSink.java │ │ │ │ └── ZooKeeperStringSerializer.java │ │ └── table │ │ │ └── descriptors │ │ │ └── psc │ │ │ ├── PscTest.java │ │ │ └── PscValidatorTest.java │ │ └── psc │ │ └── metrics │ │ └── PscMetricsUtils.java │ └── resources │ ├── context-state-serializer-1.11 │ ├── serializer-snapshot │ └── test-data │ ├── debezium-data-schema-exclude.txt │ ├── kafka-0.11-migration-kafka-producer-flink-1.8-snapshot │ ├── kafka-0.11-migration-kafka-producer-flink-1.9-snapshot │ ├── kafka-consumer-migration-test-flink1.10-empty-state-snapshot │ ├── kafka-consumer-migration-test-flink1.10-snapshot │ ├── kafka-consumer-migration-test-flink1.11-empty-state-snapshot │ ├── kafka-consumer-migration-test-flink1.11-snapshot │ ├── kafka-consumer-migration-test-flink1.4-empty-state-snapshot │ ├── kafka-consumer-migration-test-flink1.4-snapshot │ ├── kafka-consumer-migration-test-flink1.5-empty-state-snapshot │ ├── kafka-consumer-migration-test-flink1.5-snapshot │ ├── kafka-consumer-migration-test-flink1.6-empty-state-snapshot │ ├── kafka-consumer-migration-test-flink1.6-snapshot │ ├── kafka-consumer-migration-test-flink1.7-empty-state-snapshot │ ├── kafka-consumer-migration-test-flink1.7-snapshot │ ├── kafka-consumer-migration-test-flink1.8-empty-state-snapshot │ ├── kafka-consumer-migration-test-flink1.8-snapshot │ ├── kafka-consumer-migration-test-flink1.9-empty-state-snapshot │ ├── kafka-consumer-migration-test-flink1.9-snapshot │ ├── kafka-migration-kafka-producer-flink-1.10-snapshot │ ├── kafka-migration-kafka-producer-flink-1.11-snapshot │ ├── kafka-migration-kafka-producer-flink-1.8-snapshot │ ├── kafka-migration-kafka-producer-flink-1.9-snapshot │ ├── log4j2-test.xml │ ├── psc-consumer-migration-test-flink1.11-empty-state-snapshot │ ├── psc-consumer-migration-test-flink1.11-snapshot │ ├── psc-migration-psc-producer-flink-1.11-snapshot │ └── transaction-state-serializer-1.11 │ ├── serializer-snapshot │ └── test-data ├── psc-integration-test ├── README.md ├── pom.xml └── src │ └── test │ ├── java │ └── com │ │ └── pinterest │ │ ├── memq │ │ └── commons │ │ │ └── storage │ │ │ ├── LocalStorageHandler.java │ │ │ └── TestUtils.java │ │ └── psc │ │ ├── common │ │ └── TestUtils.java │ │ ├── config │ │ └── TestPscConfiguration.java │ │ ├── consumer │ │ ├── PscConsumerUtils.java │ │ ├── listener │ │ │ ├── MessageCounterListener.java │ │ │ └── SimpleConsumerRebalanceListener.java │ │ └── memq │ │ │ └── TestPscMemqConsumer.java │ │ ├── discovery │ │ ├── DiscoveryUtils.java │ │ ├── MockKafkaServersetServiceDiscoveryProvider.java │ │ ├── MockServersetServiceDiscoveryProvider.java │ │ ├── PscConsumerCreator.java │ │ ├── TestMock2ServiceDiscoveryProvider.java │ │ ├── TestMockServiceDiscoveryProvider.java │ │ └── TestServiceDiscoveryFramework.java │ │ ├── integration │ │ ├── IdentityInterceptor.java │ │ ├── KafkaCluster.java │ │ ├── TestAutoRemediation.java │ │ ├── TestMetrics.java │ │ ├── TestProducerAndConsumerWithKafkaBackend.java │ │ ├── consumer │ │ │ ├── PscConsumerRunner.java │ │ │ ├── PscConsumerRunnerResult.java │ │ │ ├── TestApiBasics.java │ │ │ ├── TestMultiKafkaBrokerBackend.java │ │ │ ├── TestMultiKafkaClusterBackends.java │ │ │ ├── TestOneKafkaBackend.java │ │ │ ├── TestOneKafkaBackendSsl.java │ │ │ └── TestSadPaths.java │ │ └── producer │ │ │ ├── PscProducerRunner.java │ │ │ └── TestOneKafkaBackend.java │ │ ├── metrics │ │ ├── MetricTags.java │ │ ├── NoOpMetricsReporter.java │ │ └── PscMetricsUtil.java │ │ ├── producer │ │ ├── PscProducerUtils.java │ │ ├── TestMultiKafkaClusterBackends.java │ │ ├── TestResetTransactionalBackendProducerWithCompatibleState.java │ │ └── TestResetTransactionalBackendProducerWithIncompatibleState.java │ │ └── utils │ │ ├── PscTestUtils.java │ │ └── Utils.java │ └── resources │ ├── generateCertificates.sh │ ├── log4j.xml │ └── mockito-extensions │ └── org.mockito.plugins.MockMaker ├── psc-logging ├── README.md ├── pom.xml └── src │ └── main │ └── java │ └── com │ └── pinterest │ └── psc │ └── logging │ ├── PscLogger.java │ ├── PscLoggerManager.java │ ├── PscLoggerType.java │ └── log4j1 │ ├── ContextDataEnabledAppender.java │ └── PscLog4j1Logger.java └── psc ├── README.md ├── pom.xml └── src ├── main ├── java │ └── com │ │ └── pinterest │ │ └── psc │ │ ├── common │ │ ├── BaseTopicUri.java │ │ ├── CloseableIterator.java │ │ ├── MessageId.java │ │ ├── OverwriteSet.java │ │ ├── PscBackendClient.java │ │ ├── PscMessage.java │ │ ├── PscMessageTags.java │ │ ├── PscPlugin.java │ │ ├── PscUtils.java │ │ ├── ServiceDiscoveryConfig.java │ │ ├── TopicRn.java │ │ ├── TopicUri.java │ │ ├── TopicUriPartition.java │ │ └── kafka │ │ │ ├── KafkaErrors.java │ │ │ ├── KafkaMessageId.java │ │ │ ├── KafkaSslUtils.java │ │ │ └── KafkaTopicUri.java │ │ ├── config │ │ ├── MetricsReporterConfiguration.java │ │ ├── PscConfiguration.java │ │ ├── PscConfigurationInternal.java │ │ ├── PscConfigurationReporter.java │ │ ├── PscConfigurationUtils.java │ │ ├── PscConsumerToBackendConsumerConfigConverter.java │ │ ├── PscConsumerToKafkaConsumerConfigConverter.java │ │ ├── PscConsumerToMemqConsumerConfigConverter.java │ │ ├── PscProducerToBackendProducerConfigConverter.java │ │ ├── PscProducerToKafkaProducerConfigConverter.java │ │ └── PscToBackendConfigConverter.java │ │ ├── consumer │ │ ├── ConsumerRebalanceListener.java │ │ ├── InterleavingPscConsumerPollMessages.java │ │ ├── OffsetCommitCallback.java │ │ ├── PscBackendConsumer.java │ │ ├── PscConsumer.java │ │ ├── PscConsumerMessage.java │ │ ├── PscConsumerPollMessageIterator.java │ │ ├── SimplePscConsumerPollMessages.java │ │ ├── ToPscMessageIteratorConverter.java │ │ ├── creation │ │ │ ├── PscBackendConsumerCreator.java │ │ │ ├── PscConsumerCreatorManager.java │ │ │ ├── PscConsumerCreatorPlugin.java │ │ │ ├── PscKafkaConsumerCreator.java │ │ │ └── PscMemqConsumerCreator.java │ │ ├── kafka │ │ │ ├── KafkaToPscMessageIteratorConverter.java │ │ │ └── PscKafkaConsumer.java │ │ ├── listener │ │ │ └── MessageListener.java │ │ └── memq │ │ │ ├── MemqIteratorAdapter.java │ │ │ ├── MemqMessageId.java │ │ │ ├── MemqMetricsHandler.java │ │ │ ├── MemqToPscMessageIteratorConverter.java │ │ │ ├── MemqTopicUri.java │ │ │ └── PscMemqConsumer.java │ │ ├── discovery │ │ ├── FallbackServiceDiscoveryProvider.java │ │ ├── Mock2ServiceDiscoveryProvider.java │ │ ├── MockServiceDiscoveryProvider.java │ │ ├── ServiceDiscoveryManager.java │ │ ├── ServiceDiscoveryPlugin.java │ │ └── ServiceDiscoveryProvider.java │ │ ├── environment │ │ ├── Ec2EnvironmentProvider.java │ │ ├── Environment.java │ │ ├── EnvironmentProvider.java │ │ ├── HostAwareEnvironmentProvider.java │ │ └── LocalEnvironmentProvider.java │ │ ├── exception │ │ ├── BackendException.java │ │ ├── ClientException.java │ │ ├── ExceptionMessage.java │ │ ├── PscException.java │ │ ├── consumer │ │ │ ├── BackendConsumerException.java │ │ │ ├── ConsumerException.java │ │ │ ├── DeserializerException.java │ │ │ ├── KeyDeserializerException.java │ │ │ ├── ValueDeserializerException.java │ │ │ └── WakeupException.java │ │ ├── handler │ │ │ └── PscErrorHandler.java │ │ ├── producer │ │ │ ├── BackendProducerException.java │ │ │ ├── KeySerializerException.java │ │ │ ├── ProducerException.java │ │ │ ├── SerializerException.java │ │ │ ├── TransactionalProducerException.java │ │ │ └── ValueSerializerException.java │ │ └── startup │ │ │ ├── ConfigurationException.java │ │ │ ├── PscStartupException.java │ │ │ ├── ServiceDiscoveryException.java │ │ │ ├── TopicRnSyntaxException.java │ │ │ └── TopicUriSyntaxException.java │ │ ├── interceptor │ │ ├── ConsumerInterceptor.java │ │ ├── ConsumerInterceptors.java │ │ ├── DeserializerInterceptor.java │ │ ├── FirstOffsetInterceptor.java │ │ ├── Interceptor.java │ │ ├── Interceptors.java │ │ ├── ProducerInterceptor.java │ │ ├── ProducerInterceptors.java │ │ ├── RawDataMetricsInterceptor.java │ │ ├── SerializerInterceptor.java │ │ ├── TimeLagInterceptor.java │ │ ├── TypePreservingInterceptor.java │ │ └── TypedDataMetricsInterceptor.java │ │ ├── metrics │ │ ├── ConsoleMetricsReporter.java │ │ ├── Metric.java │ │ ├── MetricName.java │ │ ├── MetricValueProvider.java │ │ ├── MetricsReporter.java │ │ ├── NullMetricsReporter.java │ │ ├── OpenTSDBClient.java │ │ ├── OpenTSDBReporter.java │ │ ├── PscMetricRegistryManager.java │ │ ├── PscMetricTag.java │ │ ├── PscMetricTagManager.java │ │ ├── PscMetricType.java │ │ ├── PscMetrics.java │ │ └── kafka │ │ │ ├── KafkaMetricsHandler.java │ │ │ └── KafkaUtils.java │ │ ├── producer │ │ ├── Callback.java │ │ ├── PscBackendProducer.java │ │ ├── PscProducer.java │ │ ├── PscProducerMessage.java │ │ ├── PscProducerTransactionalProperties.java │ │ ├── creation │ │ │ ├── PscBackendProducerCreator.java │ │ │ ├── PscKafkaProducerCreator.java │ │ │ ├── PscProducerCreatorManager.java │ │ │ └── PscProducerCreatorPlugin.java │ │ └── kafka │ │ │ ├── KafkaProducerTransactionalProperties.java │ │ │ └── PscKafkaProducer.java │ │ └── serde │ │ ├── ByteArrayDeserializer.java │ │ ├── ByteArraySerializer.java │ │ ├── ByteBufferDeserializer.java │ │ ├── ByteBufferSerializer.java │ │ ├── Deserializer.java │ │ ├── DoubleDeserializer.java │ │ ├── DoubleSerializer.java │ │ ├── FloatDeserializer.java │ │ ├── FloatSerializer.java │ │ ├── IntegerDeserializer.java │ │ ├── IntegerSerializer.java │ │ ├── LongDeserializer.java │ │ ├── LongSerializer.java │ │ ├── Serializer.java │ │ ├── ShortDeserializer.java │ │ ├── ShortSerializer.java │ │ ├── StringDeserializer.java │ │ ├── StringSerializer.java │ │ ├── UuidDeserializer.java │ │ └── UuidSerializer.java ├── resources-filtered │ └── psc.properties └── resources │ ├── log4j.xml │ └── psc.conf └── test ├── java └── com │ └── pinterest │ └── psc │ ├── common │ ├── TestBaseTopicUri.java │ ├── TestPscUtils.java │ ├── TestTopicRn.java │ ├── TestTopicUri.java │ ├── TestUtils.java │ └── kafka │ │ └── TestKafkaSslUtils.java │ ├── config │ └── TestPscConfiguration.java │ ├── consumer │ ├── BytesPscConsumerMessageTestUtil.java │ ├── PscConsumerMessageTestUtil.java │ ├── PscConsumerUtils.java │ ├── StringPscConsumerMessageTestUtil.java │ ├── TestInterleavingPscConsumerMessages.java │ ├── TestPscConsumer.java │ ├── TestPscConsumerBase.java │ ├── TestPscConsumerMessageListener.java │ ├── kafka │ │ ├── TestKafkaTopicUri.java │ │ └── TestPscConsumerToKafkaConsumerConfigConverter.java │ └── listener │ │ └── MessageCounterListener.java │ ├── discovery │ ├── DiscoveryUtil.java │ ├── TestFallbackServiceDiscoveryProvider.java │ └── TestServiceDiscoveryManager.java │ ├── exception │ └── TestPscErrorHandler.java │ ├── interceptor │ ├── ExceptionalInterceptor.java │ ├── ExceptionalMetricsReportingInterceptor.java │ ├── IdentityInterceptor.java │ ├── ModifierInterceptor.java │ ├── TestDeserializedDataMetricsConsumerInterceptor.java │ ├── TestPscConsumerInterceptor.java │ └── TestSerializedDataMetricsConsumerInterceptor.java │ ├── metrics │ └── MetricsUtils.java │ ├── producer │ ├── PscProducerUtils.java │ ├── TestPscProducer.java │ ├── TestPscProducerBase.java │ ├── TestTransactionalPscProducer.java │ └── kafka │ │ └── TestPscProducerToKafkaProducerConfigConverter.java │ └── serde │ └── TestSerde.java └── resources ├── kafka.keystore.jks ├── kafka.truststore.jks ├── log4j2-test.properties.alt ├── log4j2-test.xml └── mockito-extensions └── org.mockito.plugins.MockMaker /.github/CODEOWNERS: -------------------------------------------------------------------------------- 1 | # All owners 2 | * @pinterest/logging 3 | -------------------------------------------------------------------------------- /.github/workflows/maven.yml: -------------------------------------------------------------------------------- 1 | name: PSC-Java Build 2 | 3 | on: [pull_request] 4 | 5 | jobs: 6 | build: 7 | 8 | runs-on: ubuntu-latest 9 | steps: 10 | - uses: actions/checkout@v1 11 | - name: Set up JDK 1.8 12 | uses: actions/setup-java@v1 13 | with: 14 | java-version: 1.8 15 | - name: Build with Maven 16 | run: mvn -B package --file pom.xml -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | target/ 2 | pom.xml.tag 3 | pom.xml.releaseBackup 4 | pom.xml.versionsBackup 5 | pom.xml.next 6 | release.properties 7 | dependency-reduced-pom.xml 8 | buildNumber.properties 9 | .mvn/timing.properties 10 | # https://github.com/takari/maven-wrapper#usage-without-binary-jar 11 | .mvn/wrapper/maven-wrapper.jar 12 | 13 | # Eclipse m2e generated files 14 | # Eclipse Core 15 | .project 16 | # JDT-specific (Eclipse Java Development Tools) 17 | .classpath 18 | 19 | # intellij 20 | .idea 21 | *.iml 22 | 23 | # vscode 24 | .vscode 25 | -------------------------------------------------------------------------------- /ADOPTERS.md: -------------------------------------------------------------------------------- 1 | # Adopters 2 | 3 | This is an alphabetical list of people and organizations who are using this 4 | project. If you'd like to be included here, please send a Pull Request that 5 | adds your information to this file. 6 | 7 | - [Pinterest](https://www.pinterest.com/) -------------------------------------------------------------------------------- /CODE_OF_CONDUCT.md: -------------------------------------------------------------------------------- 1 | # Code of Conduct 2 | 3 | At Pinterest, we work hard to ensure that our work environment is welcoming 4 | and inclusive to as many people as possible. We are committed to creating this 5 | environment for everyone involved in our open source projects as well. We 6 | welcome all participants regardless of ability, age, ethnicity, identified 7 | gender, religion (or lack there of), sexual orientation and socioeconomic 8 | status. 9 | 10 | This code of conduct details our expectations for upholding these values. 11 | 12 | ## Good behavior 13 | 14 | We expect members of our community to exhibit good behavior including (but of 15 | course not limited to): 16 | 17 | - Using intentional and empathetic language. 18 | - Focusing on resolving instead of escalating conflict. 19 | - Providing constructive feedback. 20 | 21 | ## Unacceptable behavior 22 | 23 | Some examples of unacceptable behavior (again, this is not an exhaustive 24 | list): 25 | 26 | - Harassment, publicly or in private. 27 | - Trolling. 28 | - Sexual advances (this isn’t the place for it). 29 | - Publishing other’s personal information. 30 | - Any behavior which would be deemed unacceptable in a professional environment. 31 | 32 | ## Recourse 33 | 34 | If you are witness to or the target of unacceptable behavior, it should be 35 | reported to Pinterest at opensource-policy@pinterest.com. All reporters will 36 | be kept confidential and an appropriate response for each incident will be 37 | evaluated. 38 | 39 | If the maintainers do not uphold and enforce this code of conduct in 40 | good faith, community leadership will hold them accountable. -------------------------------------------------------------------------------- /CONTRIBUTING.md: -------------------------------------------------------------------------------- 1 | # Contributing 2 | 3 | First off, thanks for taking the time to contribute! This guide will answer 4 | some common questions about how this project works. 5 | 6 | While this is a Pinterest open source project, we welcome contributions from 7 | everyone. Regular outside contributors can become project maintainers. 8 | 9 | ## Help 10 | 11 | If you're having trouble using this project, please start by reading the [`README.md`](README.md) 12 | and searching for solutions in the existing open and closed issues. 13 | 14 | ## Security 15 | 16 | If you've found a security issue in one of our open source projects, 17 | please report it at [Bugcrowd](https://bugcrowd.com/pinterest); you may even 18 | make some money! 19 | 20 | ## Code of Conduct 21 | 22 | Please be sure to read and understand our [`CODE_OF_CONDUCT.md`](CODE_OF_CONDUCT.md). 23 | We work hard to ensure that our projects are welcoming and inclusive to as many 24 | people as possible. 25 | 26 | ## Reporting Issues 27 | 28 | If you have a bug report, please provide as much information as possible so that 29 | we can help you out: 30 | 31 | - Version of the project you're using. 32 | - Code (or even better whole projects) which reproduce the issue. 33 | - Steps which reproduce the issue. 34 | - Screenshots, GIFs or videos (if relavent). 35 | - Stack traces for crashes. 36 | - Any logs produced. 37 | 38 | ## Making Changes 39 | 40 | 1. Fork this repository to your own account 41 | 2. Make your changes and verify that tests pass 42 | 3. Commit your work and push to a new branch on your fork 43 | 4. Submit a pull request 44 | 5. Participate in the code review process by responding to feedback 45 | 46 | Once there is agreement that the code is in good shape, one of the project's 47 | maintainers will merge your contribution. 48 | 49 | To increase the chances that your pull request will be accepted: 50 | 51 | - Follow the coding style 52 | - Write tests for your changes 53 | - Write a good commit message 54 | 55 | ## License 56 | 57 | By contributing to this project, you agree that your contributions will be 58 | licensed under its [license](LICENSE). -------------------------------------------------------------------------------- /SECURITY.md: -------------------------------------------------------------------------------- 1 | # Reporting Security Issues 2 | 3 | If you discover a security issue in this project, please report it using 4 | [Bugcrowd](https://bugcrowd.com/pinterest). 5 | 6 | This will allow us to assess the risk and make a fix available before we 7 | publish a public bug report. 8 | 9 | Thanks for helping us make our software safe for everyone! -------------------------------------------------------------------------------- /docs/autoremediation.md: -------------------------------------------------------------------------------- 1 | # Auto Remediation 2 | PSC comes out-of-the-box with automated exception handling and remediation mechanisms that enable clients to self-recover from known exceptions thrown by the backend client. 3 | 4 | ## Motivation 5 | No matter how robust a client library is, client-server interactions in a distributed environment are still prone to errors. Often times, these issues can be solved via simple retries or client resets. However, in native client libraries, these exceptions are often handed off to the application layer. This leaves the handling of these exceptions to the application owners who may or may not have context on how best to address them. In many cases, they are simply not handled, causing applications to crash or restart. This increases KTLO burden for application and platform teams alike. 6 | 7 | Due to its design, PSC is uniquely positioned between the native client and application layers to handle these errors gracefully. Common PubSub client/server connectivity issues that have a known remediation on the client side, such as client reset or simple retries, can be done underneath the hood by PSC without causing disruption to the application layer. 8 | 9 | ## Usage 10 | To enable auto remediation in your PSC client, simply set this configuration: 11 | 12 | ``` 13 | psc.auto.resolution.enabled=true 14 | ``` 15 | 16 | To specify the maximum number of retries that PSC will attempt in handling a retriable exception (e.g. 5): 17 | 18 | ``` 19 | psc.auto.resolution.retry.count=5 20 | ``` -------------------------------------------------------------------------------- /docs/images/psc-architecture.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/docs/images/psc-architecture.jpg -------------------------------------------------------------------------------- /docs/quickstart.md: -------------------------------------------------------------------------------- 1 | # Quick Start Guide 2 | Here is a set of instructions on how to launch your first PSC Java client, using Kafka 2.3.1 as an example backend PubSub system. 3 | 4 | ## Build 5 | ### Prerequisites 6 | - JDK 8+ 7 | - Maven 3+ 8 | ### Clone and build 9 | ``` 10 | > git clone https://github.com/pinterest/psc.git 11 | > cd psc/java 12 | > git checkout 2.3 # use PSC 2.3 for compatibility with Kafka 2.3.1 13 | > mvn clean package -DskipTests 14 | ``` 15 | 16 | ## Start local Kafka broker 17 | To allow for PSC to publish or consume data from Kafka locally, start a local Kafka broker and create a test topic *my_test_topic*. The script [startlocalkafka.sh](../psc-examples/quickstart-scripts/startlocalkafka.sh) will achieve this for you. 18 | 19 | ``` 20 | > cd psc/java/psc-examples/quickstart-scripts 21 | 22 | # download + run ZooKeeper and Kafka 2.3.1 and create a local Kafka topic my_test_topic 23 | > ./startlocalkafka.sh # non blocking once running; to stop use ./stoplocalkafka.sh 24 | ``` 25 | 26 | ## Launching a PscProducer 27 | There is a provided [ExamplePscProducer.java](../psc-examples/src/main/java/com/pinterest/psc/example/kafka/ExamplePscProducer.java) class that launches a simple PscProducer which writes some messages to *my_test_topic* our local Kafka broker. To launch it, simply execute 28 | 29 | ``` 30 | # assuming you're still in psc-examples/quickstart-scripts directory 31 | > ./example_psc_producer.sh 32 | ``` 33 | 34 | ## Launching a PscConsumer 35 | There is a provided [ExamplePscConsumer.java](../psc-examples/src/main/java/com/pinterest/psc/example/kafka/ExamplePscConsumer.java) class that launches a simple PscConsumer which reads the messages that we just sent to *my_test_topic* our local Kafka broker. To launch it, simply execute 36 | 37 | ``` 38 | # assuming you're still in psc-examples/quickstart-scripts directory 39 | > ./example_psc_consumer.sh 40 | ``` 41 | 42 | ## Diving deeper 43 | 44 | - [Resource Names (RNs)](/docs/resourcenames.md) 45 | - [Service Discovery](/docs/servicediscovery.md) 46 | - [Auto Remediation](/docs/autoremediation.md) 47 | - [Native Kafka Client to PSC Migration](/docs/nativekafkatopscmigration.md) -------------------------------------------------------------------------------- /psc-common/README.md: -------------------------------------------------------------------------------- 1 | # PSC Common 2 | 3 | Common utility logic 4 | 5 | ## Contents 6 | This module contains some common utility logic used by PSC, such as environment detection, data comparisons / transformations, and general Java operations. -------------------------------------------------------------------------------- /psc-common/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | psc-java-oss 7 | com.pinterest.psc 8 | 2.3.0-SNAPSHOT 9 | ../pom.xml 10 | 11 | 4.0.0 12 | 13 | psc-common 14 | 15 | 16 | org.junit.jupiter 17 | junit-jupiter 18 | test 19 | 20 | 21 | 22 | 23 | 8 24 | 8 25 | 26 | 27 | -------------------------------------------------------------------------------- /psc-examples/README.md: -------------------------------------------------------------------------------- 1 | # PSC Examples 2 | 3 | Quickstart guide and examples for onboarding and migrating to PSC 4 | 5 | ## Contents 6 | This module contains several examples for onboarding and migrating to PSC. For more details, check out the [quickstart guide](/docs/quickstart.md). -------------------------------------------------------------------------------- /psc-examples/quickstart-scripts/common.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | function waitPort() { 4 | for _ in `seq 1 20`; do 5 | echo -n . 6 | if nc -z localhost $1; then 7 | break 8 | fi 9 | sleep 0.5 10 | done 11 | } -------------------------------------------------------------------------------- /psc-examples/quickstart-scripts/configs/local-region.discovery.local-cluster.serverset: -------------------------------------------------------------------------------- 1 | localhost:9092 2 | -------------------------------------------------------------------------------- /psc-examples/quickstart-scripts/configs/local-region.discovery.local-cluster_tls.serverset: -------------------------------------------------------------------------------- 1 | localhost:9093 2 | -------------------------------------------------------------------------------- /psc-examples/quickstart-scripts/example_psc_consumer.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | DIR=$(dirname "${BASH_SOURCE[0]}") 3 | cd $DIR 4 | 5 | DIR_ABS=$(pwd) 6 | 7 | java -DtempServersetDir=$DIR_ABS/configs/ -cp ../target/psc-examples-*.jar com.pinterest.psc.example.kafka.ExamplePscConsumer plaintext:/rn:kafka:dev:local-cloud_local-region::local-cluster:my_test_topic -------------------------------------------------------------------------------- /psc-examples/quickstart-scripts/example_psc_producer.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | DIR=$(dirname "${BASH_SOURCE[0]}") 3 | cd $DIR 4 | 5 | DIR_ABS=$(pwd) 6 | 7 | java -DtempServersetDir=$DIR_ABS/configs/ -cp ../target/psc-examples-*.jar com.pinterest.psc.example.kafka.ExamplePscProducer plaintext:/rn:kafka:dev:local-cloud_local-region::local-cluster:my_test_topic -------------------------------------------------------------------------------- /psc-examples/quickstart-scripts/startlocalkafka.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | DIR=$(dirname "${BASH_SOURCE[0]}") 3 | cd $DIR 4 | source ./common.sh 5 | 6 | # make target directory so the artifacts can be ignored by git 7 | mkdir -p target 8 | cd target 9 | 10 | # download kafka 11 | rm -rf kafka_2.11-2.3.1 12 | if [[ ! -f kafka_2.11-2.3.1.tgz ]]; then 13 | wget https://archive.apache.org/dist/kafka/2.3.1/kafka_2.11-2.3.1.tgz 14 | fi 15 | tar xf kafka_2.11-2.3.1.tgz 16 | 17 | 18 | cd kafka_2.11-2.3.1 19 | echo "Starting zookeeper in background" 20 | bin/zookeeper-server-start.sh -daemon config/zookeeper.properties 21 | waitPort 2181 22 | echo "Starting kafka in background" 23 | bin/kafka-server-start.sh -daemon config/server.properties 24 | waitPort 9092 25 | echo "Creating my_test_topic" 26 | bin/kafka-topics.sh --zookeeper localhost:2181 --topic my_test_topic --partitions 1 --replication-factor 1 --create 27 | -------------------------------------------------------------------------------- /psc-examples/quickstart-scripts/stoplocalkafka.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | DIR=$(dirname "${BASH_SOURCE[0]}") 3 | cd $DIR 4 | source ./common.sh 5 | 6 | waitPort 2181 7 | waitPort 9092 8 | cd target 9 | cd kafka_2.11-2.3.1 10 | echo "Deleting my_test_topic if it exists" 11 | bin/kafka-topics.sh --zookeeper localhost:2181 --topic my_test_topic --delete 12 | bin/kafka-server-stop.sh 13 | sleep 2 14 | bin/zookeeper-server-stop.sh 15 | 16 | echo "Stopped Kafka and Zookeeper servers" -------------------------------------------------------------------------------- /psc-examples/src/main/java/com/pinterest/psc/example/common/Commons.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.example.common; 2 | 3 | import java.io.ByteArrayInputStream; 4 | import java.io.ByteArrayOutputStream; 5 | import java.io.IOException; 6 | import java.io.ObjectInput; 7 | import java.io.ObjectInputStream; 8 | import java.io.ObjectOutputStream; 9 | 10 | public class Commons { 11 | public static byte[] convertToByteArray(Object object) throws IOException { 12 | if (object == null) return null; 13 | ByteArrayOutputStream bos = new ByteArrayOutputStream(); 14 | ObjectOutputStream out = new ObjectOutputStream(bos); 15 | out.writeObject(object); 16 | out.flush(); 17 | byte[] bytes = bos.toByteArray(); 18 | bos.close(); 19 | return bytes; 20 | } 21 | 22 | public static Object convertToObject(byte[] bytes) throws IOException, ClassNotFoundException { 23 | if (bytes == null) return null; 24 | ByteArrayInputStream bis = new ByteArrayInputStream(bytes); 25 | ObjectInput in = new ObjectInputStream(bis); 26 | Object object = in.readObject(); 27 | in.close(); 28 | return object; 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /psc-examples/src/main/java/com/pinterest/psc/example/common/Message.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.example.common; 2 | 3 | import java.io.Serializable; 4 | 5 | public class Message implements Serializable { 6 | private static final long serialVersionUID = -8930189990612902608L; 7 | private int id; 8 | private String text; 9 | private long timestamp; 10 | 11 | public Message(int id, String text, long timestamp) { 12 | this.id = id; 13 | this.text = text; 14 | this.timestamp = timestamp; 15 | } 16 | 17 | public int getId() { 18 | return id; 19 | } 20 | 21 | public void setId(int id) { 22 | this.id = id; 23 | } 24 | 25 | public String getText() { 26 | return text; 27 | } 28 | 29 | public void setText(String text) { 30 | this.text = text; 31 | } 32 | 33 | public long getTimestamp() { 34 | return timestamp; 35 | } 36 | 37 | public void setTimestamp(long timestamp) { 38 | this.timestamp = timestamp; 39 | } 40 | 41 | @Override 42 | public String toString() { 43 | return String.format("id: %d; text: %s; timestamp: %d", id, text, timestamp); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /psc-examples/src/main/java/com/pinterest/psc/example/common/MessageDeserializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.example.common; 2 | 3 | import com.pinterest.psc.exception.consumer.DeserializerException; 4 | import com.pinterest.psc.serde.Deserializer; 5 | 6 | import java.io.IOException; 7 | 8 | public class MessageDeserializer implements Deserializer { 9 | @Override 10 | public Message deserialize(byte[] bytes) throws DeserializerException { 11 | try { 12 | return (Message) Commons.convertToObject(bytes); 13 | } catch (IOException | ClassNotFoundException e) { 14 | throw new DeserializerException("Failed to deserialize byte array to Message object.", e); 15 | } 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /psc-examples/src/main/java/com/pinterest/psc/example/common/MessageSerializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.example.common; 2 | 3 | import com.pinterest.psc.exception.producer.SerializerException; 4 | import com.pinterest.psc.serde.Serializer; 5 | 6 | import java.io.IOException; 7 | 8 | public class MessageSerializer implements Serializer { 9 | @Override 10 | public byte[] serialize(Message message) throws SerializerException { 11 | try { 12 | return Commons.convertToByteArray(message); 13 | } catch (IOException e) { 14 | throw new SerializerException("Failed to serialize message object to byte array", e); 15 | } 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /psc-examples/src/main/java/com/pinterest/psc/example/common/TestMessageSerDe.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.example.common; 2 | 3 | import com.pinterest.psc.exception.consumer.DeserializerException; 4 | import com.pinterest.psc.exception.producer.SerializerException; 5 | 6 | public class TestMessageSerDe { 7 | public static void main(String[] args) throws SerializerException, DeserializerException { 8 | Message message = new Message(1, "My first message", System.currentTimeMillis()); 9 | byte[] serialized = new MessageSerializer().serialize(message); 10 | Message messageDeserialized = new MessageDeserializer().deserialize(serialized); 11 | assert message.getId() == messageDeserialized.getId(); 12 | assert message.getText() == messageDeserialized.getText(); 13 | assert message.getTimestamp() == messageDeserialized.getTimestamp(); 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /psc-examples/src/main/java/com/pinterest/psc/example/kafka/ExamplePscProducer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.example.kafka; 2 | 3 | import com.pinterest.psc.config.PscConfiguration; 4 | import com.pinterest.psc.exception.producer.ProducerException; 5 | import com.pinterest.psc.exception.startup.ConfigurationException; 6 | import com.pinterest.psc.logging.PscLogger; 7 | import com.pinterest.psc.producer.PscProducer; 8 | import com.pinterest.psc.producer.PscProducerMessage; 9 | import com.pinterest.psc.serde.IntegerSerializer; 10 | import com.pinterest.psc.serde.StringSerializer; 11 | 12 | public class ExamplePscProducer { 13 | 14 | private static final PscLogger logger = PscLogger.getLogger(ExamplePscProducer.class); 15 | private static final int NUM_MESSAGES = 10; 16 | 17 | public static void main(String[] args) throws ConfigurationException, ProducerException { 18 | if (args.length < 1) { 19 | logger.error("ExamplePscProducer needs one argument: topicUri"); 20 | return; 21 | } 22 | String topicUri = args[0]; 23 | 24 | PscConfiguration pscConfiguration = new PscConfiguration(); 25 | pscConfiguration.setProperty(PscConfiguration.PSC_PRODUCER_CLIENT_ID, "test-psc-producer-client"); // required 26 | pscConfiguration.setProperty(PscConfiguration.PSC_CONFIG_LOGGING_ENABLED, false); // set this to false in our example since we don't need to log the client configurations 27 | pscConfiguration.setProperty(PscConfiguration.PSC_PRODUCER_KEY_SERIALIZER, IntegerSerializer.class.getName()); 28 | pscConfiguration.setProperty(PscConfiguration.PSC_PRODUCER_VALUE_SERIALIZER, StringSerializer.class.getName()); 29 | 30 | PscProducer pscProducer = new PscProducer<>(pscConfiguration); 31 | for (int i = 0; i < NUM_MESSAGES; i++) { 32 | PscProducerMessage message = new PscProducerMessage<>(topicUri, i, "hello world " + i); 33 | logger.info("Sending message: " + message.toString() + " to topicUri " + topicUri); 34 | pscProducer.send(message); 35 | } 36 | 37 | pscProducer.close(); 38 | 39 | logger.info("ExamplePscProducer sent " + NUM_MESSAGES + " messages to " + topicUri); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /psc-examples/src/main/java/com/pinterest/psc/example/migration/consumer/Kafka.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.example.migration.consumer; 2 | 3 | import org.apache.kafka.clients.consumer.ConsumerConfig; 4 | import org.apache.kafka.clients.consumer.ConsumerRecords; 5 | import org.apache.kafka.clients.consumer.KafkaConsumer; 6 | import org.apache.kafka.common.serialization.ByteArrayDeserializer; 7 | 8 | import java.time.Duration; 9 | import java.util.Collections; 10 | import java.util.Properties; 11 | import java.util.concurrent.atomic.AtomicInteger; 12 | 13 | public class Kafka { 14 | static public void main(String[] args) { 15 | Properties properties = new Properties(); 16 | properties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); 17 | properties.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "test_consumer"); 18 | properties.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "test_group"); 19 | properties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); 20 | properties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); 21 | KafkaConsumer consumer = new KafkaConsumer<>(properties); 22 | 23 | consumer.subscribe(Collections.singleton("my_test_topic")); 24 | 25 | long current = System.currentTimeMillis(); 26 | long start = current; 27 | AtomicInteger count = new AtomicInteger(0); 28 | while (current - start < 20_000) { 29 | ConsumerRecords records = consumer.poll(Duration.ofMillis(500)); 30 | records.iterator().forEachRemaining(record -> count.incrementAndGet()); 31 | current = System.currentTimeMillis(); 32 | } 33 | System.out.printf("Consumed %d records in %d ms.%n", count.get(), current - start); 34 | consumer.close(); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /psc-examples/src/main/java/com/pinterest/psc/example/migration/consumer/Psc.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.example.migration.consumer; 2 | 3 | import com.pinterest.psc.config.PscConfiguration; 4 | import com.pinterest.psc.consumer.PscConsumer; 5 | import com.pinterest.psc.consumer.PscConsumerPollMessageIterator; 6 | import com.pinterest.psc.exception.consumer.ConsumerException; 7 | import com.pinterest.psc.exception.consumer.WakeupException; 8 | import com.pinterest.psc.exception.startup.ConfigurationException; 9 | import com.pinterest.psc.serde.ByteArrayDeserializer; 10 | 11 | import java.time.Duration; 12 | import java.util.Collections; 13 | import java.util.concurrent.atomic.AtomicInteger; 14 | 15 | public class Psc { 16 | static public void main(String[] args) throws ConsumerException, ConfigurationException, WakeupException { 17 | PscConfiguration pscConfiguration = new PscConfiguration(); 18 | pscConfiguration.setProperty(PscConfiguration.PSC_CONSUMER_CLIENT_ID, "test_consumer"); 19 | pscConfiguration.setProperty(PscConfiguration.PSC_CONSUMER_GROUP_ID, "test_group"); 20 | pscConfiguration.setProperty(PscConfiguration.PSC_CONSUMER_KEY_DESERIALIZER, ByteArrayDeserializer.class.getName()); 21 | pscConfiguration.setProperty(PscConfiguration.PSC_CONSUMER_VALUE_DESERIALIZER, ByteArrayDeserializer.class.getName()); 22 | PscConsumer consumer = new PscConsumer<>(pscConfiguration); 23 | 24 | consumer.subscribe(Collections.singleton("plaintext:/rn:kafka:dev:local-cloud_local-region::local-cluster:my_test_topic")); 25 | 26 | long current = System.currentTimeMillis(); 27 | long start = current; 28 | AtomicInteger count = new AtomicInteger(0); 29 | while (current - start < 20_000) { 30 | PscConsumerPollMessageIterator messages = consumer.poll(Duration.ofMillis(500)); 31 | messages.forEachRemaining(message -> count.incrementAndGet()); 32 | current = System.currentTimeMillis(); 33 | } 34 | System.out.printf("Consumed %d records in %d ms.%n", count.get(), current - start); 35 | consumer.close(); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /psc-examples/src/main/java/com/pinterest/psc/example/migration/producer/Kafka.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.example.migration.producer; 2 | 3 | import java.util.Properties; 4 | import java.util.concurrent.atomic.AtomicInteger; 5 | 6 | import org.apache.kafka.clients.producer.KafkaProducer; 7 | import org.apache.kafka.clients.producer.ProducerConfig; 8 | import org.apache.kafka.clients.producer.ProducerRecord; 9 | import org.apache.kafka.common.serialization.ByteArraySerializer; 10 | 11 | 12 | public class Kafka { 13 | static public void main(String[] args) { 14 | Properties properties = new Properties(); 15 | properties.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); 16 | properties.setProperty(ProducerConfig.CLIENT_ID_CONFIG, "test_producer"); 17 | properties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); 18 | properties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); 19 | 20 | KafkaProducer producer = new KafkaProducer<>(properties); 21 | 22 | long current = System.currentTimeMillis(); 23 | long start = current; 24 | AtomicInteger count = new AtomicInteger(0); 25 | while (current - start < 20_000) { 26 | ProducerRecord record = new ProducerRecord<>("my_test_topic", ("hello world " + count.getAndIncrement()).getBytes()); 27 | producer.send(record); 28 | current = System.currentTimeMillis(); 29 | } 30 | System.out.printf("Produced %d records in %d ms.%n", count.get(), current - start); 31 | producer.close(); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /psc-examples/src/main/java/com/pinterest/psc/example/migration/producer/Psc.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.example.migration.producer; 2 | 3 | import java.util.concurrent.atomic.AtomicInteger; 4 | 5 | import com.pinterest.psc.config.PscConfiguration; 6 | import com.pinterest.psc.exception.producer.ProducerException; 7 | import com.pinterest.psc.exception.startup.ConfigurationException; 8 | import com.pinterest.psc.producer.PscProducer; 9 | import com.pinterest.psc.producer.PscProducerMessage; 10 | import com.pinterest.psc.serde.ByteArraySerializer; 11 | 12 | public class Psc { 13 | 14 | static public void main(String[] args) throws ConfigurationException, ProducerException { 15 | String topicUri = "plaintext:/rn:kafka:dev:local-cloud_local-region::local-cluster:my_test_topic"; 16 | 17 | PscConfiguration pscConfiguration = new PscConfiguration(); 18 | pscConfiguration.setProperty(PscConfiguration.PSC_PRODUCER_CLIENT_ID, "test_producer"); 19 | pscConfiguration.setProperty(PscConfiguration.PSC_CONFIG_LOGGING_ENABLED, false); // set this to false in our example since we don't need to log the client configurations 20 | pscConfiguration.setProperty(PscConfiguration.PSC_PRODUCER_KEY_SERIALIZER, ByteArraySerializer.class.getName()); 21 | pscConfiguration.setProperty(PscConfiguration.PSC_PRODUCER_VALUE_SERIALIZER, ByteArraySerializer.class.getName()); 22 | 23 | PscProducer producer = new PscProducer<>(pscConfiguration); 24 | 25 | long current = System.currentTimeMillis(); 26 | long start = current; 27 | AtomicInteger count = new AtomicInteger(0); 28 | while (current - start < 20_000) { 29 | PscProducerMessage message = new PscProducerMessage<>(topicUri, ("hello world " + count.getAndIncrement()).getBytes()); 30 | producer.send(message); 31 | current = System.currentTimeMillis(); 32 | } 33 | System.out.printf("Produced %d records in %d ms.%n", count.get(), current - start); 34 | producer.close(); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /psc-examples/src/main/resources/log4j.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | 18 | 19 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 36 | 37 | 38 | 39 | 40 | 41 | 42 | 43 | 44 | 45 | 46 | 47 | 48 | 49 | -------------------------------------------------------------------------------- /psc-examples/src/main/resources/psc-override.conf: -------------------------------------------------------------------------------- 1 | #psc.metrics 2 | psc.metrics.reporter.class=com.pinterest.psc.metrics.NullMetricsReporter 3 | psc.metrics.frequency.ms=15000 4 | 5 | psc.custom.config=value 6 | -------------------------------------------------------------------------------- /psc-examples/src/main/resources/psc_kafka_consumer.conf: -------------------------------------------------------------------------------- 1 | psc.metrics.reporter.class=com.pinterest.psc.metrics.NullMetricsReporter 2 | 3 | psc.consumer.key.deserializer=com.pinterest.psc.serde.IntegerDeserializer 4 | psc.consumer.value.deserializer=com.pinterest.psc.example.common.MessageDeserializer 5 | psc.consumer.client.id=test-consumer 6 | psc.consumer.group.id=test_group1 7 | psc.consumer.poll.messages.max=2000 8 | -------------------------------------------------------------------------------- /psc-examples/src/main/resources/psc_memq_consumer.conf: -------------------------------------------------------------------------------- 1 | psc.metrics.reporter.class=com.pinterest.psc.metrics.NullMetricsReporter 2 | 3 | psc.consumer.key.deserializer=com.pinterest.psc.serde.ByteArrayDeserializer 4 | psc.consumer.value.deserializer=com.pinterest.psc.serde.ByteArrayDeserializer 5 | psc.consumer.client.id=memq-psc-consumer 6 | psc.consumer.group.id=test_memq_psc 7 | 8 | #psc.memq.notification 9 | psc.memq.notification.poll.records.max=1 10 | -------------------------------------------------------------------------------- /psc-flink-logging/README.md: -------------------------------------------------------------------------------- 1 | # PSC Flink Logging 2 | 3 | Unified logging APIs for PSC-Flink 4 | 5 | ## Contents 6 | This module contains unified Logging APIs for PSC-Flink, allowing for simple initializtion of application logging. -------------------------------------------------------------------------------- /psc-flink-logging/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | psc-java-oss 7 | com.pinterest.psc 8 | 2.3.0-SNAPSHOT 9 | ../pom.xml 10 | 11 | 4.0.0 12 | 13 | psc-flink-logging 14 | 2.3.0-SNAPSHOT 15 | 16 | 17 | 18 | org.slf4j 19 | slf4j-api 20 | 1.7.26 21 | provided 22 | 23 | 24 | com.pinterest.psc 25 | psc-common 26 | ${project.version} 27 | 28 | 29 | 30 | 31 | 8 32 | 8 33 | 34 | 35 | -------------------------------------------------------------------------------- /psc-flink-logging/src/main/java/com/pinterest/psc/logging/PscLoggerManager.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.logging; 2 | 3 | public class PscLoggerManager { 4 | protected static PscLoggerType pscLoggerType = PscLoggerType.SLF4J; 5 | 6 | public static void setPscLoggerType(PscLoggerType pscLoggerType) { 7 | PscLoggerManager.pscLoggerType = pscLoggerType; 8 | } 9 | } 10 | -------------------------------------------------------------------------------- /psc-flink-logging/src/main/java/com/pinterest/psc/logging/PscLoggerType.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.logging; 2 | 3 | public enum PscLoggerType { 4 | LOG4J, LOG4J2, LOGBACK, SLF4J 5 | } 6 | -------------------------------------------------------------------------------- /psc-flink/README.md: -------------------------------------------------------------------------------- 1 | # PSC Flink 2 | 3 | Flink-PSC connector implementation 4 | 5 | ## Contents 6 | This module contains the Flink-PSC connector implementation, which allow Apache Flink data streaming jobs to leverage the benefits of PSC. For details regarding version compatibility and migrating from Flink-Kafka, check out the [README](/README.md) and [migration doc](/docs/nativekafkatopscmigration.md). -------------------------------------------------------------------------------- /psc-flink/src/main/java/com/pinterest/flink/streaming/connectors/psc/FlinkPscErrorCode.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.pinterest.flink.streaming.connectors.psc; 19 | 20 | import org.apache.flink.annotation.PublicEvolving; 21 | 22 | /** 23 | * Error codes used in {@link FlinkPscException}. 24 | */ 25 | @PublicEvolving 26 | public enum FlinkPscErrorCode { 27 | PRODUCERS_POOL_EMPTY, 28 | EXTERNAL_ERROR 29 | } 30 | -------------------------------------------------------------------------------- /psc-flink/src/main/java/com/pinterest/flink/streaming/connectors/psc/FlinkPscException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.pinterest.flink.streaming.connectors.psc; 19 | 20 | import org.apache.flink.annotation.PublicEvolving; 21 | import org.apache.flink.util.FlinkException; 22 | 23 | /** 24 | * Exception used by {@link FlinkPscProducer} and {@link FlinkPscConsumer}. 25 | */ 26 | @PublicEvolving 27 | public class FlinkPscException extends FlinkException { 28 | 29 | private static final long serialVersionUID = 920269130311214200L; 30 | 31 | private final FlinkPscErrorCode errorCode; 32 | 33 | public FlinkPscException(FlinkPscErrorCode errorCode, String message) { 34 | super(message); 35 | this.errorCode = errorCode; 36 | } 37 | 38 | public FlinkPscException(FlinkPscErrorCode errorCode, String message, Throwable cause) { 39 | super(message, cause); 40 | this.errorCode = errorCode; 41 | } 42 | 43 | public FlinkPscErrorCode getErrorCode() { 44 | return errorCode; 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /psc-flink/src/main/java/com/pinterest/flink/streaming/connectors/psc/config/OffsetCommitMode.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.pinterest.flink.streaming.connectors.psc.config; 19 | 20 | import org.apache.flink.annotation.Internal; 21 | 22 | /** 23 | * The offset commit mode represents the behaviour of how offsets are externally committed 24 | * back to backend pubsub. 25 | * 26 | *

The exact value of this is determined at runtime in the consumer subtasks. 27 | */ 28 | @Internal 29 | public enum OffsetCommitMode { 30 | 31 | /** 32 | * Completely disable offset committing. 33 | */ 34 | DISABLED, 35 | 36 | /** 37 | * Commit offsets back to backend pubsub only when checkpoints are completed. 38 | */ 39 | ON_CHECKPOINTS, 40 | 41 | /** 42 | * Commit offsets periodically back to backend pubsub, using the auto commit functionality of internal PSC clients. 43 | */ 44 | BACKEND_PUBSUB_PERIODIC; 45 | } 46 | -------------------------------------------------------------------------------- /psc-flink/src/main/java/com/pinterest/flink/streaming/connectors/psc/internals/KeyedSerializationSchemaWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.pinterest.flink.streaming.connectors.psc.internals; 19 | 20 | import org.apache.flink.annotation.Internal; 21 | import org.apache.flink.api.common.serialization.SerializationSchema; 22 | import com.pinterest.flink.streaming.util.serialization.psc.KeyedSerializationSchema; 23 | 24 | /** 25 | * A simple wrapper for using the SerializationSchema with the KeyedSerializationSchema 26 | * interface. 27 | * 28 | * @param The type to serialize 29 | */ 30 | @Internal 31 | public class KeyedSerializationSchemaWrapper implements KeyedSerializationSchema { 32 | 33 | private static final long serialVersionUID = 1351665280744549933L; 34 | 35 | private final SerializationSchema serializationSchema; 36 | 37 | public KeyedSerializationSchemaWrapper(SerializationSchema serializationSchema) { 38 | this.serializationSchema = serializationSchema; 39 | } 40 | 41 | public SerializationSchema getSerializationSchema() { 42 | return serializationSchema; 43 | } 44 | 45 | @Override 46 | public byte[] serializeKey(T element) { 47 | return null; 48 | } 49 | 50 | @Override 51 | public byte[] serializeValue(T element) { 52 | return serializationSchema.serialize(element); 53 | } 54 | 55 | @Override 56 | public String getTargetTopic(T element) { 57 | return null; // we are never overriding the topic 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /psc-flink/src/main/java/com/pinterest/flink/streaming/connectors/psc/internals/PscCommitCallback.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.pinterest.flink.streaming.connectors.psc.internals; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | 23 | /** 24 | * A callback interface that the source operator can implement to trigger custom actions when a commit request completes, 25 | * which should normally be triggered from checkpoint complete event. 26 | */ 27 | @Internal 28 | public interface PscCommitCallback { 29 | 30 | /** 31 | * A callback method the user can implement to provide asynchronous handling of commit request completion. 32 | * This method will be called when the commit request sent to the server has been acknowledged without error. 33 | */ 34 | void onSuccess(); 35 | 36 | /** 37 | * A callback method the user can implement to provide asynchronous handling of commit request failure. 38 | * This method will be called when the commit request failed. 39 | * 40 | * @param cause PSC commit failure cause returned by PSC client 41 | */ 42 | void onException(Throwable cause); 43 | } 44 | -------------------------------------------------------------------------------- /psc-flink/src/main/java/com/pinterest/flink/streaming/connectors/psc/internals/SourceContextWatermarkOutputAdapter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.pinterest.flink.streaming.connectors.psc.internals; 19 | 20 | import org.apache.flink.api.common.eventtime.Watermark; 21 | import org.apache.flink.api.common.eventtime.WatermarkOutput; 22 | import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; 23 | 24 | /** 25 | * A {@link WatermarkOutput} that forwards calls to a {@link 26 | * SourceContext}. 27 | */ 28 | public class SourceContextWatermarkOutputAdapter implements WatermarkOutput { 29 | private final SourceContext sourceContext; 30 | 31 | public SourceContextWatermarkOutputAdapter(SourceContext sourceContext) { 32 | this.sourceContext = sourceContext; 33 | } 34 | 35 | @Override 36 | public void emitWatermark(Watermark watermark) { 37 | sourceContext.emitWatermark( 38 | new org.apache.flink.streaming.api.watermark.Watermark(watermark.getTimestamp())); 39 | } 40 | 41 | @Override 42 | public void markIdle() { 43 | sourceContext.markAsTemporarilyIdle(); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /psc-flink/src/main/java/com/pinterest/flink/streaming/connectors/psc/internals/metrics/PscMetricMutableWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.pinterest.flink.streaming.connectors.psc.internals.metrics; 20 | 21 | import com.pinterest.psc.metrics.Metric; 22 | import org.apache.flink.annotation.Internal; 23 | import org.apache.flink.metrics.Gauge; 24 | 25 | /** 26 | * Gauge for getting the current value of a PSC metric. 27 | */ 28 | @Internal 29 | public class PscMetricMutableWrapper implements Gauge { 30 | private Metric pscMetric; 31 | 32 | public PscMetricMutableWrapper(Metric metric) { 33 | this.pscMetric = metric; 34 | } 35 | 36 | @Override 37 | public Double getValue() { 38 | return (Double) pscMetric.metricValue(); 39 | } 40 | 41 | public void setPscMetric(Metric pscMetric) { 42 | this.pscMetric = pscMetric; 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /psc-flink/src/main/java/com/pinterest/flink/streaming/connectors/psc/internals/metrics/PscMetricWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.pinterest.flink.streaming.connectors.psc.internals.metrics; 20 | 21 | import com.pinterest.psc.metrics.Metric; 22 | import org.apache.flink.annotation.Internal; 23 | import org.apache.flink.metrics.Gauge; 24 | 25 | /** 26 | * Gauge for getting the current value of a PSC metric. 27 | */ 28 | @Internal 29 | public class PscMetricWrapper implements Gauge { 30 | private final Metric pscMetric; 31 | 32 | public PscMetricWrapper(Metric metric) { 33 | this.pscMetric = metric; 34 | } 35 | 36 | @Override 37 | public Double getValue() { 38 | return (Double) pscMetric.metricValue(); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /psc-flink/src/main/java/com/pinterest/flink/streaming/connectors/psc/shuffle/StreamPscShuffleSink.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.pinterest.flink.streaming.connectors.psc.shuffle; 19 | 20 | import org.apache.flink.annotation.Internal; 21 | import org.apache.flink.streaming.api.operators.StreamOperator; 22 | import org.apache.flink.streaming.api.operators.StreamSink; 23 | import org.apache.flink.streaming.api.watermark.Watermark; 24 | 25 | /** 26 | * A customized {@link StreamOperator} for executing {@link FlinkPscShuffleProducer} that handle 27 | * both elements and watermarks. If the shuffle sink is determined to be useful to other sinks in the future, 28 | * we should abstract this operator to data stream api. For now, we keep the operator this way to avoid 29 | * public interface change. 30 | */ 31 | @Internal 32 | class StreamPscShuffleSink extends StreamSink { 33 | 34 | public StreamPscShuffleSink(FlinkPscShuffleProducer flinkPscShuffleProducer) { 35 | super(flinkPscShuffleProducer); 36 | } 37 | 38 | @Override 39 | public void processWatermark(Watermark mark) throws Exception { 40 | super.processWatermark(mark); 41 | ((FlinkPscShuffleProducer) userFunction).invoke(mark); 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /psc-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | org.apache.flink.streaming.connectors.kafka.table.KafkaDynamicTableFactory 17 | com.pinterest.flink.streaming.connectors.psc.table.PscDynamicTableFactory 18 | -------------------------------------------------------------------------------- /psc-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | org.apache.flink.streaming.connectors.kafka.KafkaTableSourceSinkFactory 17 | com.pinterest.flink.streaming.connectors.psc.PscTableSourceSinkFactory 18 | -------------------------------------------------------------------------------- /psc-flink/src/test/java/com/pinterest/flink/streaming/connectors/psc/BasicPscConsumerTest.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.flink.streaming.connectors.psc; 2 | 3 | import org.apache.flink.api.common.restartstrategy.RestartStrategies; 4 | import org.apache.flink.api.common.serialization.SimpleStringSchema; 5 | import org.apache.flink.streaming.api.datastream.DataStream; 6 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 7 | import org.apache.flink.streaming.api.functions.sink.DiscardingSink; 8 | 9 | import java.util.Properties; 10 | 11 | public class BasicPscConsumerTest extends PscConsumerTestBaseWithKafkaAsPubSub { 12 | 13 | public static void printProperties(Properties prop) { 14 | for (Object key : prop.keySet()) { 15 | System.out.println(key + ": " + prop.getProperty(key.toString())); 16 | } 17 | } 18 | 19 | // @Test 20 | public void testSimpleConsumptionCount() throws Exception { 21 | final int parallelism = 1; 22 | final int recordsInEachPartition = 50; 23 | StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); 24 | env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); 25 | env.setParallelism(parallelism); 26 | env.enableCheckpointing(200); 27 | 28 | int numPartitions = parallelism; 29 | String topicName = "test_simple_count"; 30 | createTestTopic(topicName, numPartitions, 1); 31 | pscTestEnvWithKafka.produceToKafka(topicName, recordsInEachPartition, numPartitions, "hello"); 32 | 33 | Properties readProps = new Properties(); 34 | readProps.putAll(standardPscConsumerConfiguration); 35 | printProperties(readProps); 36 | 37 | DataStream stream = env 38 | .addSource(pscTestEnvWithKafka.getPscConsumer(topicName, new SimpleStringSchema(), readProps)); 39 | stream.addSink(new DiscardingSink()); 40 | env.execute("test_simple_count"); 41 | 42 | deleteTestTopic(topicName); 43 | } 44 | 45 | } 46 | -------------------------------------------------------------------------------- /psc-flink/src/test/java/com/pinterest/flink/streaming/connectors/psc/PscProducerAtLeastOnceITCase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.pinterest.flink.streaming.connectors.psc; 20 | 21 | import org.junit.BeforeClass; 22 | 23 | /** 24 | * IT cases for the {@link FlinkPscProducer}. 25 | */ 26 | @SuppressWarnings("serial") 27 | public class PscProducerAtLeastOnceITCase extends PscProducerTestBase { 28 | 29 | @BeforeClass 30 | public static void prepare() throws Exception { 31 | PscProducerTestBase.prepare(); 32 | ((PscTestEnvironmentWithKafkaAsPubSubImpl) pscTestEnvWithKafka).setProducerSemantic(FlinkPscProducer.Semantic.AT_LEAST_ONCE); 33 | } 34 | 35 | @Override 36 | public void testExactlyOnceRegularSink() throws Exception { 37 | // disable test for at least once semantic 38 | } 39 | 40 | @Override 41 | public void testExactlyOnceCustomOperator() throws Exception { 42 | // disable test for at least once semantic 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /psc-flink/src/test/java/com/pinterest/flink/streaming/connectors/psc/PscProducerExactlyOnceITCase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.pinterest.flink.streaming.connectors.psc; 20 | 21 | import org.junit.BeforeClass; 22 | import org.junit.Test; 23 | 24 | /** 25 | * IT cases for the {@link FlinkPscProducer}. 26 | */ 27 | @SuppressWarnings("serial") 28 | public class PscProducerExactlyOnceITCase extends PscProducerTestBase { 29 | @BeforeClass 30 | public static void prepare() throws Exception { 31 | PscProducerTestBase.prepare(); 32 | ((PscTestEnvironmentWithKafkaAsPubSubImpl) pscTestEnvWithKafka).setProducerSemantic(FlinkPscProducer.Semantic.EXACTLY_ONCE); 33 | } 34 | 35 | @Test 36 | public void testMultipleSinkOperators() throws Exception { 37 | testExactlyOnce(false, 2); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /psc-flink/src/test/java/com/pinterest/flink/streaming/connectors/psc/PscTestBaseWithFlinkWithKafkaAsPubSub.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.pinterest.flink.streaming.connectors.psc; 19 | 20 | import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; 21 | import org.apache.flink.test.util.MiniClusterWithClientResource; 22 | import org.junit.ClassRule; 23 | 24 | /** 25 | * The base for the PSC tests with Flink's MiniCluster and Kafka as backend pubsub. 26 | */ 27 | @SuppressWarnings("serial") 28 | public abstract class PscTestBaseWithFlinkWithKafkaAsPubSub extends PscTestBaseWithKafkaAsPubSub { 29 | 30 | protected static final int NUM_TMS = 1; 31 | 32 | protected static final int TM_SLOTS = 8; 33 | 34 | @ClassRule 35 | public static MiniClusterWithClientResource flink = new MiniClusterWithClientResource( 36 | new MiniClusterResourceConfiguration.Builder() 37 | .setConfiguration(getFlinkConfiguration()) 38 | .setNumberTaskManagers(NUM_TMS) 39 | .setNumberSlotsPerTaskManager(TM_SLOTS) 40 | .build()); 41 | } 42 | -------------------------------------------------------------------------------- /psc-flink/src/test/java/com/pinterest/flink/streaming/connectors/psc/table/PscTableITCase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.pinterest.flink.streaming.connectors.psc.table; 20 | 21 | /** 22 | * IT cases for PSC for Table API & SQL. 23 | */ 24 | //public class PscTableITCase extends PscTableTestBase { 25 | // 26 | // @Override 27 | // public String factoryIdentifier() { 28 | // return PscDynamicTableFactory.IDENTIFIER; 29 | // } 30 | // 31 | // @Override 32 | // public String pscVersion() { 33 | // return PscValidator.CONNECTOR_VERSION_VALUE_UNIVERSAL; 34 | // } 35 | //} 36 | -------------------------------------------------------------------------------- /psc-flink/src/test/java/com/pinterest/flink/streaming/connectors/psc/testutils/ClusterCommunicationUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.pinterest.flink.streaming.connectors.psc.testutils; 20 | 21 | import org.apache.flink.api.common.JobID; 22 | import org.apache.flink.client.program.ClusterClient; 23 | import org.apache.flink.runtime.client.JobStatusMessage; 24 | 25 | import java.util.Collection; 26 | import java.util.List; 27 | import java.util.stream.Collectors; 28 | 29 | /** 30 | * Utilities for communicating with a cluster through a {@link ClusterClient}. 31 | */ 32 | public class ClusterCommunicationUtils { 33 | 34 | public static void waitUntilJobIsRunning(ClusterClient client) throws Exception { 35 | while (getRunningJobs(client).isEmpty()) { 36 | Thread.sleep(50); 37 | } 38 | } 39 | 40 | public static void waitUntilNoJobIsRunning(ClusterClient client) throws Exception { 41 | while (!getRunningJobs(client).isEmpty()) { 42 | Thread.sleep(50); 43 | } 44 | } 45 | 46 | public static List getRunningJobs(ClusterClient client) throws Exception { 47 | Collection statusMessages = client.listJobs().get(); 48 | return statusMessages.stream() 49 | .filter(status -> !status.getJobState().isGloballyTerminalState()) 50 | .map(JobStatusMessage::getJobId) 51 | .collect(Collectors.toList()); 52 | } 53 | 54 | private ClusterCommunicationUtils() { 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /psc-flink/src/test/java/com/pinterest/flink/streaming/connectors/psc/testutils/FakeStandardPscProducerConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.pinterest.flink.streaming.connectors.psc.testutils; 20 | 21 | import com.pinterest.psc.config.PscConfiguration; 22 | import com.pinterest.psc.serde.ByteArraySerializer; 23 | 24 | import java.util.Properties; 25 | 26 | /** 27 | * Test configuration for a kafka producer. 28 | */ 29 | public class FakeStandardPscProducerConfig { 30 | 31 | public static Properties get() { 32 | Properties p = new Properties(); 33 | p.setProperty(PscConfiguration.PSC_PRODUCER_KEY_SERIALIZER, ByteArraySerializer.class.getName()); 34 | p.setProperty(PscConfiguration.PSC_PRODUCER_VALUE_SERIALIZER, ByteArraySerializer.class.getName()); 35 | return p; 36 | } 37 | 38 | } 39 | -------------------------------------------------------------------------------- /psc-flink/src/test/java/com/pinterest/flink/streaming/connectors/psc/testutils/PartitionValidatingMapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.pinterest.flink.streaming.connectors.psc.testutils; 20 | 21 | import org.apache.flink.api.common.functions.MapFunction; 22 | 23 | import java.util.HashSet; 24 | import java.util.Set; 25 | 26 | /** 27 | * {@link MapFunction} that verifies that he partitioning is identical. 28 | */ 29 | public class PartitionValidatingMapper implements MapFunction { 30 | 31 | private static final long serialVersionUID = 1088381231244959088L; 32 | 33 | /* the partitions from which this function received data */ 34 | private final Set myPartitions = new HashSet<>(); 35 | 36 | private final int numPartitions; 37 | private final int maxPartitions; 38 | 39 | public PartitionValidatingMapper(int numPartitions, int maxPartitions) { 40 | this.numPartitions = numPartitions; 41 | this.maxPartitions = maxPartitions; 42 | } 43 | 44 | @Override 45 | public Integer map(Integer value) throws Exception { 46 | // validate that the partitioning is identical 47 | int partition = value % numPartitions; 48 | myPartitions.add(partition); 49 | if (myPartitions.size() > maxPartitions) { 50 | throw new Exception("Error: Elements from too many different partitions: " + myPartitions 51 | + ". Expect elements only from " + maxPartitions + " partitions"); 52 | } 53 | return value; 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /psc-flink/src/test/java/com/pinterest/flink/streaming/connectors/psc/testutils/ThrottledMapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.pinterest.flink.streaming.connectors.psc.testutils; 20 | 21 | import org.apache.flink.api.common.functions.MapFunction; 22 | 23 | /** 24 | * An identity map function that sleeps between elements, throttling the 25 | * processing speed. 26 | * 27 | * @param The type mapped. 28 | */ 29 | public class ThrottledMapper implements MapFunction { 30 | 31 | private static final long serialVersionUID = 467008933767159126L; 32 | 33 | private final int sleep; 34 | 35 | public ThrottledMapper(int sleep) { 36 | this.sleep = sleep; 37 | } 38 | 39 | @Override 40 | public T map(T value) throws Exception { 41 | Thread.sleep(this.sleep); 42 | return value; 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /psc-flink/src/test/java/com/pinterest/flink/streaming/connectors/psc/testutils/Tuple2FlinkPartitioner.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.pinterest.flink.streaming.connectors.psc.testutils; 20 | 21 | import com.pinterest.flink.streaming.connectors.psc.partitioner.FlinkPscPartitioner; 22 | import org.apache.flink.api.java.tuple.Tuple2; 23 | 24 | /** 25 | * Special partitioner that uses the first field of a 2-tuple as the partition, 26 | * and that expects a specific number of partitions. 27 | */ 28 | public class Tuple2FlinkPartitioner extends FlinkPscPartitioner> { 29 | private static final long serialVersionUID = -3589898230375281549L; 30 | 31 | private final int expectedPartitions; 32 | 33 | public Tuple2FlinkPartitioner(int expectedPartitions) { 34 | this.expectedPartitions = expectedPartitions; 35 | } 36 | 37 | @Override 38 | public int partition(Tuple2 next, byte[] key, byte[] value, String targetTopic, int[] partitions) { 39 | if (partitions.length != expectedPartitions) { 40 | throw new IllegalArgumentException("Expected " + expectedPartitions + " partitions"); 41 | } 42 | 43 | return next.f0; 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /psc-flink/src/test/java/com/pinterest/flink/streaming/connectors/psc/testutils/ZooKeeperStringSerializer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.pinterest.flink.streaming.connectors.psc.testutils; 20 | 21 | import org.I0Itec.zkclient.serialize.ZkSerializer; 22 | import org.apache.flink.configuration.ConfigConstants; 23 | 24 | /** 25 | * Simple ZooKeeper serializer for Strings. 26 | */ 27 | public class ZooKeeperStringSerializer implements ZkSerializer { 28 | 29 | @Override 30 | public byte[] serialize(Object data) { 31 | if (data instanceof String) { 32 | return ((String) data).getBytes(ConfigConstants.DEFAULT_CHARSET); 33 | } else { 34 | throw new IllegalArgumentException("ZooKeeperStringSerializer can only serialize strings."); 35 | } 36 | } 37 | 38 | @Override 39 | public Object deserialize(byte[] bytes) { 40 | if (bytes == null) { 41 | return null; 42 | } else { 43 | return new String(bytes, ConfigConstants.DEFAULT_CHARSET); 44 | } 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /psc-flink/src/test/resources/context-state-serializer-1.11/serializer-snapshot: -------------------------------------------------------------------------------- 1 | torg.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$ContextStateSerializer$ContextStateSerializerSnapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/context-state-serializer-1.11/test-data: -------------------------------------------------------------------------------- 1 | 123456789 -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.8-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.8-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.9-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-0.11-migration-kafka-producer-flink-1.9-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.10-empty-state-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.10-empty-state-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.10-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.10-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.11-empty-state-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.11-empty-state-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.11-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.11-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.4-empty-state-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.4-empty-state-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.4-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.4-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.5-empty-state-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.5-empty-state-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.5-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.5-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.6-empty-state-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.6-empty-state-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.6-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.6-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.7-empty-state-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.7-empty-state-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.7-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.7-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.8-empty-state-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.8-empty-state-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.8-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.8-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.9-empty-state-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.9-empty-state-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.9-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-consumer-migration-test-flink1.9-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-migration-kafka-producer-flink-1.10-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-migration-kafka-producer-flink-1.10-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-migration-kafka-producer-flink-1.11-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-migration-kafka-producer-flink-1.11-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-migration-kafka-producer-flink-1.8-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-migration-kafka-producer-flink-1.8-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/kafka-migration-kafka-producer-flink-1.9-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/kafka-migration-kafka-producer-flink-1.9-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/log4j2-test.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | 18 | 19 | -------------------------------------------------------------------------------- /psc-flink/src/test/resources/psc-consumer-migration-test-flink1.11-empty-state-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/psc-consumer-migration-test-flink1.11-empty-state-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/psc-consumer-migration-test-flink1.11-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/psc-consumer-migration-test-flink1.11-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/psc-migration-psc-producer-flink-1.11-snapshot: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/psc-migration-psc-producer-flink-1.11-snapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/transaction-state-serializer-1.11/serializer-snapshot: -------------------------------------------------------------------------------- 1 | |org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer$TransactionStateSerializer$TransactionStateSerializerSnapshot -------------------------------------------------------------------------------- /psc-flink/src/test/resources/transaction-state-serializer-1.11/test-data: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc-flink/src/test/resources/transaction-state-serializer-1.11/test-data -------------------------------------------------------------------------------- /psc-integration-test/README.md: -------------------------------------------------------------------------------- 1 | # PSC Integration Test 2 | 3 | Integration test suite for PSC 4 | 5 | ## Contents 6 | This module contains an extensive suite of integration tests that verify the end-to-end functionality of PSC. To run integration tests: 7 | 8 | ``` 9 | > mvn clean install 10 | > cd psc-integration-test 11 | > mvn clean test 12 | ``` -------------------------------------------------------------------------------- /psc-integration-test/src/test/java/com/pinterest/psc/common/TestUtils.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.common; 2 | 3 | import com.pinterest.psc.metrics.NullMetricsReporter; 4 | 5 | public class TestUtils { 6 | public static final String DEFAULT_METRICS_REPORTER = NullMetricsReporter.class.getName(); 7 | 8 | public static TopicUriPartition getFinalizedTopicUriPartition(TopicUri topicUri, int partition) { 9 | return new TopicUriPartition(topicUri, partition); 10 | } 11 | } 12 | -------------------------------------------------------------------------------- /psc-integration-test/src/test/java/com/pinterest/psc/consumer/PscConsumerUtils.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.consumer; 2 | 3 | import com.pinterest.psc.config.PscConfiguration; 4 | import com.pinterest.psc.config.PscConfigurationInternal; 5 | import com.pinterest.psc.exception.consumer.ConsumerException; 6 | import com.pinterest.psc.exception.startup.ConfigurationException; 7 | import com.pinterest.psc.metrics.PscMetricRegistryManager; 8 | 9 | import java.util.Collection; 10 | import java.util.Map; 11 | 12 | public class PscConsumerUtils { 13 | @SuppressWarnings("unchecked") 14 | public static Map getBackendConsumerConfiguration(PscConsumer pscConsumer) { 15 | return pscConsumer.getBackendConsumerConfigurationPerTopicUri(); 16 | } 17 | 18 | public static Collection getBackendConsumersOf(PscConsumer pscConsumer) { 19 | return pscConsumer.getBackendConsumers(); 20 | } 21 | 22 | public static void resetBackendConsumer(PscConsumer pscConsumer, PscBackendConsumer pscBackendConsumer) 23 | throws ConfigurationException, ConsumerException { 24 | pscConsumer.reset(pscBackendConsumer); 25 | } 26 | 27 | public static PscMetricRegistryManager getMetricRegistryManager(PscConsumer pscConsumer) { 28 | return pscConsumer.getPscMetricRegistryManager(); 29 | } 30 | 31 | public static PscConfigurationInternal getPscConfigurationInternal(PscConsumer pscConsumer) { 32 | return pscConsumer.getPscConfiguration(); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /psc-integration-test/src/test/java/com/pinterest/psc/consumer/listener/MessageCounterListener.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.consumer.listener; 2 | 3 | import com.pinterest.psc.consumer.PscConsumer; 4 | import com.pinterest.psc.consumer.PscConsumerMessage; 5 | 6 | public class MessageCounterListener implements MessageListener { 7 | private int handleCallCounter = 0; 8 | 9 | @Override 10 | public void handle(PscConsumer consumer, PscConsumerMessage message) { 11 | ++handleCallCounter; 12 | } 13 | 14 | public int getHandleCallCounter() { 15 | return handleCallCounter; 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /psc-integration-test/src/test/java/com/pinterest/psc/consumer/listener/SimpleConsumerRebalanceListener.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.consumer.listener; 2 | 3 | import com.pinterest.psc.common.TopicUriPartition; 4 | import com.pinterest.psc.consumer.ConsumerRebalanceListener; 5 | import com.pinterest.psc.logging.PscLogger; 6 | 7 | import java.util.Collection; 8 | 9 | public class SimpleConsumerRebalanceListener implements ConsumerRebalanceListener { 10 | private int revoked = -1; 11 | private int assigned = -1; 12 | private static final PscLogger logger = PscLogger.getLogger(SimpleConsumerRebalanceListener.class); 13 | 14 | @Override 15 | public void onPartitionsRevoked(Collection partitions) { 16 | this.revoked = partitions.size(); 17 | logger.info("Partitions revoked: " + revoked); 18 | } 19 | 20 | @Override 21 | public void onPartitionsAssigned(Collection partitions) { 22 | this.assigned = partitions.size(); 23 | logger.info("Partitions assigned: " + assigned); 24 | } 25 | 26 | public int getRevoked() { return this.revoked; } 27 | public int getAssigned() { return this.assigned; } 28 | } 29 | -------------------------------------------------------------------------------- /psc-integration-test/src/test/java/com/pinterest/psc/discovery/DiscoveryUtils.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.discovery; 2 | 3 | public class DiscoveryUtils { 4 | public static void overrideServersetPath(String newPath) { 5 | MockServersetServiceDiscoveryProvider.overrideServersetPath(newPath); 6 | } 7 | } 8 | -------------------------------------------------------------------------------- /psc-integration-test/src/test/java/com/pinterest/psc/integration/IdentityInterceptor.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.integration; 2 | 3 | import com.pinterest.psc.common.MessageId; 4 | import com.pinterest.psc.consumer.PscConsumerMessage; 5 | import com.pinterest.psc.interceptor.TypePreservingInterceptor; 6 | 7 | import java.util.Collection; 8 | 9 | public class IdentityInterceptor extends TypePreservingInterceptor { 10 | public int onConsumeCounter = 0; 11 | public int onCommitCounter = 0; 12 | 13 | @Override 14 | public PscConsumerMessage onConsume(PscConsumerMessage message) { 15 | onConsumeCounter++; 16 | return message; 17 | } 18 | 19 | @Override 20 | public void onCommit(Collection messageIds) { 21 | messageIds.forEach(messageId -> onCommitCounter++); 22 | } 23 | } 24 | -------------------------------------------------------------------------------- /psc-integration-test/src/test/java/com/pinterest/psc/integration/KafkaCluster.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.integration; 2 | 3 | import com.google.common.io.Files; 4 | import com.pinterest.psc.discovery.DiscoveryUtils; 5 | 6 | import java.io.File; 7 | import java.io.IOException; 8 | 9 | public class KafkaCluster { 10 | private static final String TEST_SERVERSET_PATH = "/tmp/"; 11 | private String transport, region, cluster; 12 | private int port; 13 | 14 | public KafkaCluster(String transport, String region, String cluster, int port) throws IOException { 15 | this.transport = transport; 16 | this.region = region; 17 | this.cluster = cluster; 18 | this.port = port; 19 | createServersetFile(); 20 | } 21 | 22 | public String getTransport() { 23 | return transport; 24 | } 25 | 26 | public String getRegion() { 27 | return region; 28 | } 29 | 30 | public String getCluster() { 31 | return cluster; 32 | } 33 | 34 | public int getPort() { 35 | return port; 36 | } 37 | 38 | private void createServersetFile() throws IOException { 39 | Files.write(getBootstrap().getBytes(), new File(getServersetFileName(false))); 40 | new File(getServersetFileName(false)).deleteOnExit(); 41 | } 42 | 43 | public void createTlsServersetFile() throws IOException { 44 | Files.write(getBootstrap().getBytes(), new File(getServersetFileName(true))); 45 | new File(getServersetFileName(true)).deleteOnExit(); 46 | } 47 | 48 | public String getBootstrap() { 49 | return "localhost:" + port; 50 | } 51 | 52 | private String getServersetFileName(boolean tls) { 53 | DiscoveryUtils.overrideServersetPath(TEST_SERVERSET_PATH); 54 | if (tls) { 55 | return TEST_SERVERSET_PATH + String.format("%s.discovery.%s_tls.serverset", region, cluster); 56 | } 57 | return TEST_SERVERSET_PATH + String.format("%s.discovery.%s.serverset", region, cluster); 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /psc-integration-test/src/test/java/com/pinterest/psc/integration/consumer/PscConsumerRunnerResult.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.integration.consumer; 2 | 3 | import com.pinterest.psc.common.MessageId; 4 | import com.pinterest.psc.common.TopicUriPartition; 5 | import com.pinterest.psc.consumer.PscConsumerMessage; 6 | 7 | import java.util.List; 8 | import java.util.Set; 9 | 10 | public class PscConsumerRunnerResult { 11 | Set assignment; 12 | List> messages; 13 | Set committed; 14 | 15 | public PscConsumerRunnerResult( 16 | Set assignment, 17 | List> messages, 18 | Set committed 19 | ) { 20 | this.assignment = assignment; 21 | this.messages = messages; 22 | this.committed = committed; 23 | } 24 | 25 | public Set getAssignment() { 26 | return assignment; 27 | } 28 | 29 | public List> getMessages() { 30 | return messages; 31 | } 32 | 33 | public Set getCommitted() { 34 | return committed; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /psc-integration-test/src/test/java/com/pinterest/psc/metrics/MetricTags.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.metrics; 2 | 3 | import java.util.Map; 4 | 5 | public class MetricTags { 6 | private final String hostname; 7 | private final String hostIp; 8 | private final String locality; 9 | private final String instanceType; 10 | private final String processId; 11 | private final long threadId; 12 | private final String uri; 13 | private final String project; 14 | private final String version; 15 | private final Map others; 16 | 17 | public MetricTags(String hostname, String hostIp, String locality, String instanceType, String processId, long threadId, String uri, String project, String version, Map others) { 18 | this.hostname = hostname; 19 | this.hostIp = hostIp; 20 | this.locality = locality; 21 | this.instanceType = instanceType; 22 | this.processId = processId; 23 | this.threadId = threadId; 24 | this.uri = uri; 25 | this.project = project; 26 | this.version = version; 27 | this.others = others; 28 | } 29 | 30 | public String getHostname() { 31 | return hostname; 32 | } 33 | 34 | public String getHostIp() { 35 | return hostIp; 36 | } 37 | 38 | public String getLocality() { 39 | return locality; 40 | } 41 | 42 | public String getInstanceType() { 43 | return instanceType; 44 | } 45 | 46 | public String getProcessId() { 47 | return processId; 48 | } 49 | 50 | public long getThreadId() { 51 | return threadId; 52 | } 53 | 54 | public String getUri() { 55 | return uri; 56 | } 57 | 58 | public String getProject() { 59 | return project; 60 | } 61 | 62 | public String getVersion() { 63 | return version; 64 | } 65 | 66 | public Map getOthers() { 67 | return others; 68 | } 69 | 70 | @Override 71 | public String toString() { 72 | return String.format("Proj-%s, Th-%d", project, threadId); 73 | } 74 | } 75 | -------------------------------------------------------------------------------- /psc-integration-test/src/test/java/com/pinterest/psc/metrics/PscMetricsUtil.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.metrics; 2 | 3 | import com.codahale.metrics.ScheduledReporter; 4 | 5 | import java.util.Map; 6 | 7 | public class PscMetricsUtil { 8 | 9 | public static boolean isInitializationError(PscMetricRegistryManager pscMetricRegistryManager) { 10 | return pscMetricRegistryManager.isInitializationError(); 11 | } 12 | 13 | public static Map getPscMetricReporterMap(PscMetricRegistryManager pscMetricRegistryManager) { 14 | return pscMetricRegistryManager.getPscMetricReporterMap(); 15 | } 16 | 17 | public static void cleanup(PscMetricRegistryManager pscMetricRegistryManager) { 18 | pscMetricRegistryManager.cleanup(); 19 | pscMetricRegistryManager.clearCurrentThreadMetricMap(); 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /psc-integration-test/src/test/java/com/pinterest/psc/producer/PscProducerUtils.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.producer; 2 | 3 | import com.pinterest.psc.config.PscConfigurationInternal; 4 | import com.pinterest.psc.consumer.PscConsumer; 5 | import com.pinterest.psc.exception.producer.ProducerException; 6 | import com.pinterest.psc.exception.startup.ConfigurationException; 7 | import com.pinterest.psc.metrics.PscMetricRegistryManager; 8 | 9 | import java.util.Collection; 10 | 11 | public class PscProducerUtils { 12 | public static Collection getBackendProducersOf(PscProducer pscProducer) { 13 | return pscProducer.getBackendProducers(); 14 | } 15 | 16 | public static void resetBackendProducer(PscProducer pscProducer, PscBackendProducer pscBackendProducer) 17 | throws ProducerException, ConfigurationException { 18 | pscProducer.reset(pscBackendProducer); 19 | } 20 | 21 | public static PscProducerTransactionalProperties initTransactions(PscProducer pscProducer, String topicUri) 22 | throws ConfigurationException, ProducerException { 23 | return pscProducer.initTransactions(topicUri); 24 | } 25 | 26 | public static PscMetricRegistryManager getMetricRegistryManager(PscProducer pscProducer) { 27 | return pscProducer.getPscMetricRegistryManager(); 28 | } 29 | 30 | public static PscConfigurationInternal getPscConfigurationInternal(PscProducer pscProducer) { 31 | return pscProducer.getPscConfiguration(); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /psc-integration-test/src/test/java/com/pinterest/psc/utils/Utils.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.utils; 2 | 3 | import java.util.ArrayList; 4 | import java.util.HashSet; 5 | import java.util.List; 6 | import java.util.Random; 7 | import java.util.Set; 8 | 9 | public class Utils { 10 | public static Set getRandomElements(List list, int count) throws Exception { 11 | if (count < 0) return null; 12 | if (count == 0) return new HashSet<>(); 13 | int size = list.size(); 14 | if (count > size) 15 | throw new Exception(String.format("count > list size: %d > %d", count, size)); 16 | 17 | Set randomIndices = new HashSet<>(); 18 | Random random = new Random(); 19 | while (randomIndices.size() < count) 20 | randomIndices.add(random.nextInt(size)); 21 | 22 | Set randomElements = new HashSet<>(); 23 | randomIndices.forEach(index -> randomElements.add(list.get(index))); 24 | return randomElements; 25 | } 26 | 27 | public static Set getRandomNumbers(int min, int max, int count) throws Exception { 28 | List list = new ArrayList<>(); 29 | for (int i = min; i <= max; ++i) 30 | list.add(i); 31 | return getRandomElements(list, count); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /psc-integration-test/src/test/resources/log4j.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | -------------------------------------------------------------------------------- /psc-integration-test/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker: -------------------------------------------------------------------------------- 1 | mock-maker-inline -------------------------------------------------------------------------------- /psc-logging/README.md: -------------------------------------------------------------------------------- 1 | # PSC Logging 2 | 3 | Unified logging APIs for PSC 4 | 5 | ## Contents 6 | This module contains unified Logging APIs for PSC, allowing for simple initializtion of application logging. -------------------------------------------------------------------------------- /psc-logging/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | psc-java-oss 7 | com.pinterest.psc 8 | 2.3.0-SNAPSHOT 9 | ../pom.xml 10 | 11 | 4.0.0 12 | 13 | psc-logging 14 | 15 | 16 | 8 17 | 8 18 | 19 | 20 | 21 | 22 | com.pinterest.psc 23 | psc-common 24 | ${project.version} 25 | 26 | 27 | log4j 28 | log4j 29 | 1.2.17 30 | 31 | 32 | log4j 33 | apache-log4j-extras 34 | 1.2.17 35 | 36 | 37 | net.logstash.log4j 38 | jsonevent-layout 39 | 1.7 40 | 41 | 42 | net.minidev 43 | json-smart 44 | 45 | 46 | 47 | 48 | net.minidev 49 | json-smart 50 | 2.4.9 51 | 52 | 53 | 54 | -------------------------------------------------------------------------------- /psc-logging/src/main/java/com/pinterest/psc/logging/PscLoggerManager.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.logging; 2 | 3 | public class PscLoggerManager { 4 | protected static PscLoggerType pscLoggerType = PscLoggerType.LOG4J; 5 | 6 | public static void setPscLoggerType(PscLoggerType pscLoggerType) { 7 | PscLoggerManager.pscLoggerType = pscLoggerType; 8 | } 9 | } 10 | -------------------------------------------------------------------------------- /psc-logging/src/main/java/com/pinterest/psc/logging/PscLoggerType.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.logging; 2 | 3 | public enum PscLoggerType { 4 | LOG4J, LOG4J2, LOGBACK 5 | } 6 | -------------------------------------------------------------------------------- /psc-logging/src/main/java/com/pinterest/psc/logging/log4j1/ContextDataEnabledAppender.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.logging.log4j1; 2 | 3 | import org.apache.log4j.FileAppender; 4 | import org.apache.log4j.spi.LoggingEvent; 5 | 6 | import java.text.SimpleDateFormat; 7 | import java.util.Date; 8 | 9 | public class ContextDataEnabledAppender extends FileAppender { 10 | private static final SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss ZZZ"); 11 | private static final Date date = new Date(); 12 | 13 | @Override 14 | protected void subAppend(LoggingEvent event) { 15 | date.setTime(event.getTimeStamp()); 16 | event.setProperty("ts", sdf.format(date)); 17 | super.subAppend(event); 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /psc/README.md: -------------------------------------------------------------------------------- 1 | # PSC Core 2 | 3 | Core PSC components and classes 4 | 5 | ## Contents 6 | This module contains the core PSC classes, such as [PscConsumer](/psc/src/main/java/com/pinterest/psc/consumer/PscConsumer.java), [PscProducer](/psc/src/main/java/com/pinterest/psc/consumer/PscProducer.java), [ServiceDiscovery](/psc/src/main/java/com/pinterest/psc/discovery), and more. -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/common/CloseableIterator.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.common; 2 | 3 | import java.io.Closeable; 4 | import java.util.Iterator; 5 | 6 | public interface CloseableIterator extends Iterator, Closeable { 7 | } 8 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/common/OverwriteSet.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.common; 2 | 3 | import java.util.Collection; 4 | import java.util.HashSet; 5 | import java.util.Set; 6 | 7 | public class OverwriteSet { 8 | private Set set = new HashSet<>(4); 9 | 10 | public Set put(Collection items) { 11 | set.clear(); 12 | set.addAll(items); 13 | return set; 14 | } 15 | 16 | public Set put(Object item) { 17 | set.clear(); 18 | set.add(item); 19 | return set; 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/common/PscMessage.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.common; 2 | 3 | public class PscMessage { 4 | public static final String PSC_MESSAGE_HEADER_PSC_PRODUCE_TIMESTAMP = "psc.produce.timestamp"; 5 | public static final String PSC_MESSAGE_HEADER_KEY_SIZE_BYTES = "psc.message.key.size.bytes"; 6 | public static final String PSC_MESSAGE_HEADER_VALUE_SIZE_BYTES = "psc.message.value.size.bytes"; 7 | } 8 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/common/PscMessageTags.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.common; 2 | 3 | public interface PscMessageTags { 4 | } 5 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/common/PscPlugin.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.common; 2 | 3 | import com.pinterest.psc.config.PscConfiguration; 4 | 5 | public interface PscPlugin { 6 | default void configure(PscConfiguration pscConfiguration) { 7 | } 8 | } 9 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/common/kafka/KafkaMessageId.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.common.kafka; 2 | 3 | import com.pinterest.psc.common.MessageId; 4 | import com.pinterest.psc.common.TopicUriPartition; 5 | 6 | public class KafkaMessageId extends MessageId { 7 | private static final long serialVersionUID = 2382070822439582943L; 8 | 9 | public KafkaMessageId(TopicUriPartition topicUriPartition, long offset, long timestamp, int serializedKeySizeBytes, int serializedValueSizeBytes) { 10 | super(topicUriPartition, offset, timestamp, serializedKeySizeBytes, serializedValueSizeBytes); 11 | } 12 | 13 | public KafkaMessageId(TopicUriPartition topicUriPartition, long offset, long timestamp) { 14 | super(topicUriPartition, offset, timestamp); 15 | } 16 | 17 | public KafkaMessageId(TopicUriPartition topicUriPartition, long offset) { 18 | super(topicUriPartition, offset); 19 | } 20 | 21 | public KafkaMessageId(MessageId messageId) { 22 | super(messageId.getTopicUriPartition(), messageId.getOffset(), messageId.getTimestamp()); 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/common/kafka/KafkaTopicUri.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.common.kafka; 2 | 3 | import com.pinterest.psc.common.BaseTopicUri; 4 | import com.pinterest.psc.common.PscUtils; 5 | import com.pinterest.psc.common.TopicUri; 6 | import com.pinterest.psc.exception.startup.TopicUriSyntaxException; 7 | 8 | public class KafkaTopicUri extends BaseTopicUri { 9 | public static final String PLAINTEXT_PROTOCOL = "plaintext"; 10 | public static final String SECURE_PROTOCOL = "secure"; 11 | 12 | public KafkaTopicUri(TopicUri baseTopicUri) { 13 | super(baseTopicUri); 14 | } 15 | 16 | public static KafkaTopicUri validate(TopicUri baseTopicUri) throws TopicUriSyntaxException { 17 | if (baseTopicUri.getTopicRn() == null) 18 | throw new TopicUriSyntaxException(baseTopicUri.getTopicUriAsString(), "Missing topic RN in topic URI"); 19 | if (!baseTopicUri.getTopicRn().getService().equals(PscUtils.BACKEND_TYPE_KAFKA)) 20 | throw new TopicUriSyntaxException(baseTopicUri.getTopicUriAsString(), "Not a Kafka URI"); 21 | if (baseTopicUri.getProtocol() != null && 22 | !baseTopicUri.getProtocol().equals(PLAINTEXT_PROTOCOL) && 23 | !baseTopicUri.getProtocol().equals(SECURE_PROTOCOL)) { 24 | throw new TopicUriSyntaxException( 25 | baseTopicUri.getTopicUriAsString(), "Invalid protocol in Kafka Topic URI: '" + 26 | baseTopicUri.getProtocol() + "'. " + "Supported protocols are '" + SECURE_PROTOCOL + 27 | "' and '" + PLAINTEXT_PROTOCOL + "'" 28 | ); 29 | } 30 | return new KafkaTopicUri(baseTopicUri); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/config/MetricsReporterConfiguration.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.config; 2 | 3 | public class MetricsReporterConfiguration { 4 | private final boolean reportingEnabled; 5 | private final String reporterClass; 6 | private final int reporterParallelism; 7 | private final String host; 8 | private final int port; 9 | private final int frequencyMs; 10 | 11 | public MetricsReporterConfiguration( 12 | boolean reportingEnabled, String reporterClass, int reporterParallelism, String host, int port, int frequencyMs 13 | ) { 14 | this.reportingEnabled = reportingEnabled; 15 | this.reporterClass = reporterClass; 16 | this.reporterParallelism = reporterParallelism; 17 | this.host = host; 18 | this.port = port; 19 | this.frequencyMs = frequencyMs; 20 | } 21 | 22 | public boolean isReportingEnabled() { 23 | return reportingEnabled; 24 | } 25 | 26 | public String getReporterClass() { 27 | return reporterClass; 28 | } 29 | 30 | public int getReporterParallelism() { 31 | return reporterParallelism; 32 | } 33 | 34 | public String getHost() { 35 | return host; 36 | } 37 | 38 | public int getPort() { 39 | return port; 40 | } 41 | 42 | public int getFrequencyMs() { 43 | return frequencyMs; 44 | } 45 | 46 | @Override 47 | public String toString() { 48 | return String.format("reportingEnabled=%b, reporterClass=%s, reporterParallelism=%d, host=%s, port=%d, frequencyMs=%d", 49 | reportingEnabled, reporterClass, reporterParallelism, host, port, frequencyMs); 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/config/PscConfigurationUtils.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.config; 2 | 3 | import com.pinterest.psc.exception.startup.ConfigurationException; 4 | import com.pinterest.psc.metrics.PscMetricRegistryManager; 5 | import org.apache.commons.configuration2.Configuration; 6 | import org.apache.commons.configuration2.ConfigurationConverter; 7 | import org.apache.commons.configuration2.ConfigurationUtils; 8 | 9 | import java.util.Properties; 10 | 11 | public class PscConfigurationUtils { 12 | 13 | public static void copy(Configuration configuration, PscConfiguration dstConfiguration) { 14 | ConfigurationUtils.copy(configuration, dstConfiguration); 15 | } 16 | 17 | public static PscConfigurationInternal propertiesToPscConfigurationInternal(Properties props, String clientType) { 18 | PscConfiguration pscConfiguration = new PscConfiguration(); 19 | props.keySet().stream().map(Object::toString).filter(key -> key.startsWith("psc.")).forEach(key -> pscConfiguration.setProperty(key, props.get(key))); 20 | try { 21 | return new PscConfigurationInternal(pscConfiguration, clientType, true, false); 22 | } catch (ConfigurationException exception) { 23 | throw new IllegalArgumentException(exception); 24 | } 25 | } 26 | 27 | public static Properties pscConfigurationInternalToProperties(PscConfigurationInternal pscConfigurationInternal) { 28 | return ConfigurationConverter.getProperties(pscConfigurationInternal.getConfiguration()); 29 | } 30 | 31 | } 32 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/config/PscConsumerToMemqConsumerConfigConverter.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.config; 2 | 3 | import java.util.HashMap; 4 | import java.util.Map; 5 | 6 | public class PscConsumerToMemqConsumerConfigConverter 7 | extends PscConsumerToBackendConsumerConfigConverter { 8 | @Override 9 | protected Map getConfigConverterMap() { 10 | return new HashMap() { 11 | private static final long serialVersionUID = 1L; 12 | 13 | { 14 | put(PscConfiguration.POLL_MESSAGES_MAX, "notification.max.poll.records"); 15 | put(PscConfiguration.OFFSET_AUTO_RESET, "notification.auto.offset.reset"); 16 | put(PscConfiguration.METADATA_AGE_MAX_MS, "notification.metadata.max.age.ms"); 17 | put(PscConfiguration.COMMIT_AUTO_ENABLED, "notification.enable.auto.commit"); 18 | } 19 | }; 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/config/PscProducerToBackendProducerConfigConverter.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.config; 2 | 3 | import com.pinterest.psc.common.TopicUri; 4 | import org.apache.kafka.clients.producer.ProducerConfig; 5 | 6 | import java.util.Properties; 7 | 8 | public abstract class PscProducerToBackendProducerConfigConverter extends PscToBackendConfigConverter { 9 | 10 | public Properties convert(PscConfigurationInternal pscConfigurationInternal, TopicUri topicUri) { 11 | Properties properties = new Properties(); 12 | PscConfiguration pscProducerConfiguration = pscConfigurationInternal.getProducerConfiguration(); 13 | 14 | // first build backend configs for properties in the above map 15 | PSC_TO_BACKEND_CONFIG_CONVERTER_MAP.forEach((pscConfig, backendConfig) -> { 16 | if (pscProducerConfiguration.containsKey(pscConfig)) 17 | properties.setProperty(backendConfig, pscProducerConfiguration.getString(pscConfig)); 18 | }); 19 | 20 | // next move all other configs over (pass through) only if PSC did not provide a value for them 21 | pscProducerConfiguration.getKeys().forEachRemaining(key -> { 22 | if (!PSC_TO_BACKEND_CONFIG_CONVERTER_MAP.containsKey(key) && !properties.containsKey(key)) 23 | properties.setProperty(key, pscProducerConfiguration.getString(key)); 24 | }); 25 | 26 | // distinguish different producer objects created on the same psc client instance. 27 | // this helps avoid issues such as metric tag clashes in the backend metrics processing. 28 | // example: javax.management.InstanceAlreadyExistsException: kafka.producer:type=app-info,id=test-producer 29 | // when a psc producer launches multiple kafka producers 30 | String configuredClientId = properties.getProperty(ProducerConfig.CLIENT_ID_CONFIG); 31 | properties.setProperty(ProducerConfig.CLIENT_ID_CONFIG, 32 | configuredClientId + String.format("-%s-%s", topicUri.getBackend(), topicUri.getCluster()) 33 | ); 34 | 35 | return properties; 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/config/PscToBackendConfigConverter.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.config; 2 | 3 | import com.pinterest.psc.common.TopicUri; 4 | 5 | import java.util.Map; 6 | import java.util.Properties; 7 | 8 | public abstract class PscToBackendConfigConverter { 9 | // a map of PSC Config to backend config; sample entry 10 | // poll.messages.max -> max.poll.records 11 | protected final Map PSC_TO_BACKEND_CONFIG_CONVERTER_MAP = getConfigConverterMap(); 12 | 13 | // interface to extend for each backend converter class 14 | protected abstract Map getConfigConverterMap(); 15 | 16 | protected abstract Properties convert(PscConfigurationInternal pscConfigurationInternal, TopicUri topicUri); 17 | } 18 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/consumer/ConsumerRebalanceListener.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.consumer; 2 | 3 | import java.util.Collection; 4 | import com.pinterest.psc.common.TopicUriPartition; 5 | 6 | public interface ConsumerRebalanceListener { 7 | void onPartitionsRevoked(Collection partitions); 8 | void onPartitionsAssigned(Collection partitions); 9 | } 10 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/consumer/OffsetCommitCallback.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.consumer; 2 | 3 | import com.pinterest.psc.common.MessageId; 4 | import com.pinterest.psc.common.TopicUriPartition; 5 | 6 | import java.util.Map; 7 | 8 | public interface OffsetCommitCallback { 9 | void onCompletion(Map offsets, Exception exception); 10 | } 11 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/consumer/PscConsumerPollMessageIterator.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.consumer; 2 | 3 | import com.pinterest.psc.common.CloseableIterator; 4 | import com.pinterest.psc.common.TopicUriPartition; 5 | 6 | import java.io.IOException; 7 | import java.util.Collections; 8 | import java.util.Set; 9 | 10 | public abstract class PscConsumerPollMessageIterator implements 11 | CloseableIterator> { 12 | public PscConsumerPollMessageIterator() { 13 | } 14 | 15 | public static PscConsumerPollMessageIterator emptyIterator() { 16 | return new PscConsumerPollMessageIterator() { 17 | 18 | @Override 19 | public boolean hasNext() { 20 | return false; 21 | } 22 | 23 | @Override 24 | public PscConsumerMessage next() { 25 | return null; 26 | } 27 | 28 | @Override 29 | public Set getTopicUriPartitions() { 30 | return Collections.emptySet(); 31 | } 32 | 33 | @Override 34 | public PscConsumerPollMessageIterator iteratorFor(TopicUriPartition topicUriPartition) { 35 | return null; 36 | } 37 | 38 | @Override 39 | public void close() throws IOException { 40 | 41 | } 42 | }; 43 | } 44 | 45 | /** 46 | * This API can be used to access the portion of this message iterator that is associated with the given topic 47 | * URI partition. Note that traversing the sub-iterator does not impact this (main) iterator. 48 | * 49 | * @param topicUriPartition the topic URI partition to filter messages on 50 | * @return a message iterator with only messages from the given topic URI partition. 51 | */ 52 | public abstract PscConsumerPollMessageIterator iteratorFor(TopicUriPartition topicUriPartition); 53 | 54 | /** 55 | * This API can be used to inspect all topic URI partitions associated with the returned message iterator. 56 | * @return a set of topic URI partitions from which messages exist in this message iterator. 57 | */ 58 | public abstract Set getTopicUriPartitions(); 59 | } 60 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/consumer/ToPscMessageIteratorConverter.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.consumer; 2 | 3 | import com.pinterest.psc.interceptor.ConsumerInterceptors; 4 | 5 | public abstract class ToPscMessageIteratorConverter extends PscConsumerPollMessageIterator { 6 | protected final ConsumerInterceptors consumerInterceptors; 7 | 8 | public ToPscMessageIteratorConverter(ConsumerInterceptors consumerInterceptors) { 9 | this.consumerInterceptors = consumerInterceptors; 10 | } 11 | 12 | public final PscConsumerMessage next() { 13 | return consumerInterceptors.onConsume(getNextBackendMessage()); 14 | } 15 | 16 | protected abstract PscConsumerMessage getNextBackendMessage(); 17 | } -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/consumer/creation/PscConsumerCreatorPlugin.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.consumer.creation; 2 | 3 | import java.lang.annotation.ElementType; 4 | import java.lang.annotation.Retention; 5 | import java.lang.annotation.RetentionPolicy; 6 | import java.lang.annotation.Target; 7 | 8 | @Retention(RetentionPolicy.RUNTIME) 9 | @Target({ElementType.TYPE}) 10 | public @interface PscConsumerCreatorPlugin { 11 | String backend(); 12 | } -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/consumer/listener/MessageListener.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.consumer.listener; 2 | 3 | import com.pinterest.psc.consumer.PscConsumer; 4 | import com.pinterest.psc.consumer.PscConsumerMessage; 5 | 6 | public interface MessageListener { 7 | void handle(PscConsumer consumer, PscConsumerMessage message); 8 | } 9 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/consumer/memq/MemqIteratorAdapter.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.consumer.memq; 2 | 3 | import com.pinterest.psc.common.CloseableIterator; 4 | 5 | import java.io.IOException; 6 | import java.util.function.Consumer; 7 | 8 | public class MemqIteratorAdapter implements CloseableIterator { 9 | 10 | com.pinterest.memq.commons.CloseableIterator original; 11 | 12 | public MemqIteratorAdapter(com.pinterest.memq.commons.CloseableIterator original) { 13 | this.original = original; 14 | } 15 | 16 | @Override 17 | public void close() throws IOException { 18 | original.close(); 19 | } 20 | 21 | @Override 22 | public boolean hasNext() { 23 | return original.hasNext(); 24 | } 25 | 26 | @Override 27 | public T next() { 28 | return original.next(); 29 | } 30 | 31 | @Override 32 | public void remove() { 33 | original.remove(); 34 | } 35 | 36 | @Override 37 | public void forEachRemaining(Consumer action) { 38 | original.forEachRemaining(action); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/consumer/memq/MemqMessageId.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.consumer.memq; 2 | 3 | import com.pinterest.psc.common.MessageId; 4 | import com.pinterest.psc.common.TopicUriPartition; 5 | 6 | public class MemqMessageId extends MessageId { 7 | 8 | private static final long serialVersionUID = 7724281495471377020L; 9 | 10 | public MemqMessageId(TopicUriPartition topicUriPartition, long offset, long timestamp, int serializedKeySizeBytes, int serializedValueSizeBytes) { 11 | super(topicUriPartition, offset, timestamp, serializedKeySizeBytes, serializedValueSizeBytes); 12 | } 13 | 14 | public MemqMessageId(TopicUriPartition topicUriPartition, long offset, long timestamp) { 15 | super(topicUriPartition, offset, timestamp); 16 | } 17 | 18 | public MemqMessageId(TopicUriPartition topicUriPartition, long offset) { 19 | super(topicUriPartition, offset); 20 | } 21 | 22 | public MemqMessageId(MessageId messageId) { 23 | super(messageId.getTopicUriPartition(), messageId.getOffset(), messageId.getTimestamp()); 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/consumer/memq/MemqTopicUri.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.consumer.memq; 2 | 3 | import com.pinterest.psc.common.BaseTopicUri; 4 | import com.pinterest.psc.common.PscUtils; 5 | import com.pinterest.psc.common.TopicUri; 6 | import com.pinterest.psc.exception.startup.TopicUriSyntaxException; 7 | 8 | public class MemqTopicUri extends BaseTopicUri { 9 | public static final String PLAINTEXT_PROTOCOL = "plaintext"; 10 | public static final String SECURE_PROTOCOL = "secure"; 11 | 12 | MemqTopicUri(TopicUri topicUri) { 13 | super(topicUri); 14 | } 15 | 16 | public static MemqTopicUri validate(TopicUri baseTopicUri) throws TopicUriSyntaxException { 17 | if (baseTopicUri.getTopicRn() == null) 18 | throw new TopicUriSyntaxException(baseTopicUri.getTopicUriAsString(), 19 | "Missing topic RN in Memq URI"); 20 | if (!baseTopicUri.getTopicRn().getService().equals(PscUtils.BACKEND_TYPE_MEMQ)) 21 | throw new TopicUriSyntaxException(baseTopicUri.getTopicUriAsString(), "Not a MemQ URI"); 22 | if (baseTopicUri.getProtocol() != null && 23 | !baseTopicUri.getProtocol().equals(PLAINTEXT_PROTOCOL) && 24 | !baseTopicUri.getProtocol().equals(SECURE_PROTOCOL)) { 25 | throw new TopicUriSyntaxException( 26 | baseTopicUri.getTopicUriAsString(), "Invalid protocol in MemQ Topic URI: '" + 27 | baseTopicUri.getProtocol() + "'. " + "Supported protocols are '" + SECURE_PROTOCOL + 28 | "' and '" + PLAINTEXT_PROTOCOL + "'" 29 | ); 30 | } 31 | return new MemqTopicUri(baseTopicUri); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/discovery/MockServiceDiscoveryProvider.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.discovery; 2 | 3 | import com.pinterest.psc.common.ServiceDiscoveryConfig; 4 | import com.pinterest.psc.common.TopicUri; 5 | import com.pinterest.psc.config.PscConfiguration; 6 | import com.pinterest.psc.environment.Environment; 7 | import com.pinterest.psc.logging.PscLogger; 8 | 9 | @ServiceDiscoveryPlugin(priority = 101) 10 | public class MockServiceDiscoveryProvider implements ServiceDiscoveryProvider { 11 | private static final PscLogger logger = PscLogger.getLogger(MockServiceDiscoveryProvider.class); 12 | private String connectionUrl; 13 | private String securityProtocol; 14 | 15 | public MockServiceDiscoveryProvider() { 16 | } 17 | 18 | @Override 19 | public void configure(PscConfiguration pscConfiguration) { 20 | connectionUrl = pscConfiguration.getString("connection.url"); 21 | securityProtocol = pscConfiguration.getString("security.protocol"); 22 | } 23 | 24 | @Override 25 | public ServiceDiscoveryConfig getConfig(Environment env, TopicUri topicUri) { 26 | if (connectionUrl == null || securityProtocol == null) 27 | return null; 28 | 29 | return new ServiceDiscoveryConfig() 30 | .setServiceDiscoveryProvider(this) 31 | .setConnect(connectionUrl) 32 | .setSecurityProtocol(securityProtocol); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/discovery/ServiceDiscoveryPlugin.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.discovery; 2 | 3 | import java.lang.annotation.ElementType; 4 | import java.lang.annotation.Retention; 5 | import java.lang.annotation.RetentionPolicy; 6 | import java.lang.annotation.Target; 7 | 8 | @Retention(RetentionPolicy.RUNTIME) 9 | @Target({ElementType.TYPE}) 10 | public @interface ServiceDiscoveryPlugin { 11 | int priority(); 12 | } 13 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/discovery/ServiceDiscoveryProvider.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.discovery; 2 | 3 | import com.pinterest.psc.common.PscPlugin; 4 | import com.pinterest.psc.common.ServiceDiscoveryConfig; 5 | import com.pinterest.psc.common.TopicUri; 6 | import com.pinterest.psc.environment.Environment; 7 | 8 | public interface ServiceDiscoveryProvider extends PscPlugin { 9 | ServiceDiscoveryConfig getConfig(Environment env, TopicUri topicUri); 10 | } 11 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/environment/Ec2EnvironmentProvider.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.environment; 2 | 3 | import software.amazon.awssdk.core.exception.SdkClientException; 4 | import software.amazon.awssdk.regions.internal.util.EC2MetadataUtils; 5 | 6 | public class Ec2EnvironmentProvider extends EnvironmentProvider { 7 | private static final String PROJECT_URI = "PROJECT_URI"; 8 | private static final String PROJECT = "PROJECT"; 9 | private static final String DEPLOYMENT_STAGE = "DEPLOYMENT_STAGE"; 10 | 11 | @Override 12 | public String getInstanceId() { 13 | try { 14 | return EC2MetadataUtils.getInstanceId(); 15 | } catch (SdkClientException e) { 16 | return Environment.INFO_NOT_AVAILABLE; 17 | } 18 | } 19 | 20 | @Override 21 | public String getInstanceType() { 22 | try { 23 | return EC2MetadataUtils.getInstanceType(); 24 | } catch (SdkClientException e) { 25 | return Environment.INFO_NOT_AVAILABLE; 26 | } 27 | } 28 | 29 | @Override 30 | public String getIpAddress() { 31 | try { 32 | return EC2MetadataUtils.getPrivateIpAddress(); 33 | } catch (SdkClientException e) { 34 | return Environment.INFO_NOT_AVAILABLE; 35 | } 36 | } 37 | 38 | @Override 39 | public String getLocality() { 40 | try { 41 | return EC2MetadataUtils.getAvailabilityZone(); 42 | } catch (SdkClientException e) { 43 | return Environment.INFO_NOT_AVAILABLE; 44 | } 45 | } 46 | 47 | @Override 48 | public String getRegion() { 49 | try { 50 | return EC2MetadataUtils.getEC2InstanceRegion(); 51 | } catch (SdkClientException e) { 52 | return Environment.INFO_NOT_AVAILABLE; 53 | } 54 | } 55 | 56 | @Override 57 | public String getDeploymentStage() { 58 | return System.getenv(DEPLOYMENT_STAGE); 59 | } 60 | 61 | @Override 62 | public String getProjectUri() { 63 | return System.getenv(PROJECT_URI); 64 | } 65 | 66 | @Override 67 | public String getProject() { 68 | return System.getenv(PROJECT); 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/environment/EnvironmentProvider.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.environment; 2 | 3 | import com.pinterest.psc.common.PscPlugin; 4 | 5 | /** 6 | * Environment provider for PSC. This class can be 7 | * extended to change the behavior of environment provider. 8 | */ 9 | public abstract class EnvironmentProvider implements PscPlugin { 10 | 11 | protected Environment environment = new Environment(); 12 | 13 | public EnvironmentProvider() { 14 | environment.setHostname(getHostname()); 15 | environment.setIpAddress(getIpAddress()); 16 | environment.setInstanceId(getInstanceId()); 17 | environment.setInstanceType(getInstanceType()); 18 | environment.setLocality(getLocality()); 19 | environment.setRegion(getRegion()); 20 | environment.setDeploymentStage(getDeploymentStage()); 21 | } 22 | 23 | public String getHostname() { 24 | return Environment.DEFAULT_HOSTNAME; 25 | } 26 | 27 | public abstract String getIpAddress(); 28 | 29 | public abstract String getInstanceId(); 30 | 31 | public abstract String getInstanceType(); 32 | 33 | public abstract String getLocality(); 34 | 35 | public abstract String getRegion(); 36 | 37 | public abstract String getDeploymentStage(); 38 | 39 | public abstract String getProjectUri(); 40 | 41 | public abstract String getProject(); 42 | 43 | public Environment getEnvironment() { 44 | return environment; 45 | } 46 | } -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/environment/HostAwareEnvironmentProvider.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.environment; 2 | 3 | import com.pinterest.psc.common.PscUtils; 4 | import com.pinterest.psc.logging.PscLogger; 5 | 6 | public class HostAwareEnvironmentProvider extends EnvironmentProvider { 7 | private static final PscLogger logger = PscLogger.getLogger(HostAwareEnvironmentProvider.class); 8 | 9 | private static class LazyHolder { 10 | private final static EnvironmentProvider delegate; 11 | 12 | static { 13 | if (PscUtils.isEc2Host()) { 14 | logger.info("EC2 host detected; using EC2 environment provider."); 15 | delegate = new Ec2EnvironmentProvider(); 16 | } else { 17 | logger.info("No EC2 host detected; using local environment provider."); 18 | delegate = new LocalEnvironmentProvider(); 19 | } 20 | } 21 | } 22 | 23 | private EnvironmentProvider getDelegate() { 24 | return LazyHolder.delegate; 25 | } 26 | 27 | @Override 28 | public String getIpAddress() { 29 | return getDelegate().getIpAddress(); 30 | } 31 | 32 | @Override 33 | public String getInstanceId() { 34 | return getDelegate().getInstanceId(); 35 | } 36 | 37 | @Override 38 | public String getInstanceType() { 39 | return getDelegate().getInstanceType(); 40 | } 41 | 42 | @Override 43 | public String getLocality() { 44 | return getDelegate().getLocality(); 45 | } 46 | 47 | @Override 48 | public String getRegion() { 49 | return getDelegate().getRegion(); 50 | } 51 | 52 | @Override 53 | public String getDeploymentStage() { 54 | return getDelegate().getDeploymentStage(); 55 | } 56 | 57 | @Override 58 | public String getProjectUri() { 59 | return getDelegate().getProjectUri(); 60 | } 61 | 62 | @Override 63 | public String getProject() { 64 | return getDelegate().getProject(); 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/environment/LocalEnvironmentProvider.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.environment; 2 | 3 | import com.pinterest.psc.common.PscCommon; 4 | 5 | public class LocalEnvironmentProvider extends EnvironmentProvider { 6 | public static final String LOCAL_ENV = "local-dev"; 7 | 8 | @Override 9 | public String getInstanceId() { 10 | return LOCAL_ENV; 11 | } 12 | 13 | @Override 14 | public String getInstanceType() { 15 | return LOCAL_ENV; 16 | } 17 | 18 | @Override 19 | public String getIpAddress() { 20 | return PscCommon.getHostIp(); 21 | } 22 | 23 | @Override 24 | public String getLocality() { 25 | return LOCAL_ENV; 26 | } 27 | 28 | @Override 29 | public String getRegion() { 30 | return LOCAL_ENV; 31 | } 32 | 33 | @Override 34 | public String getDeploymentStage() { 35 | return LOCAL_ENV; 36 | } 37 | 38 | @Override 39 | public String getProjectUri() { 40 | return LOCAL_ENV; 41 | } 42 | 43 | @Override 44 | public String getProject() { 45 | return LOCAL_ENV; 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/BackendException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception; 2 | 3 | public interface BackendException { 4 | 5 | public String getBackend(); 6 | } 7 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/ClientException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception; 2 | 3 | /** 4 | * This is a superclass for PSC consumer or producer exceptions. 5 | */ 6 | public class ClientException extends PscException { 7 | private static final long serialVersionUID = 1L; 8 | 9 | public ClientException(String message) { 10 | super(message); 11 | } 12 | 13 | public ClientException(String message, Throwable cause) { 14 | super(message, cause); 15 | } 16 | 17 | public ClientException(Throwable cause) { 18 | super(cause); 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/ExceptionMessage.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception; 2 | 3 | import com.pinterest.psc.common.TopicUriPartition; 4 | 5 | import java.util.Set; 6 | import java.util.stream.Collectors; 7 | 8 | public class ExceptionMessage { 9 | public static final String MULTITHREADED_EXCEPTION = "PscConsumer is not safe for multi-threaded access."; 10 | public static final String ALREADY_CLOSED_EXCEPTION = "The PSC client instance has already been closed."; 11 | public static final String ITERATOR_OUT_OF_ELEMENTS = "No more elements exist in the iterator."; 12 | public static String NO_SUBSCRIPTION_ASSIGNMENT = 13 | "PSC consumer is not subscribed to the URI or assigned the partition"; 14 | 15 | 16 | public static String TOPIC_URI_UNSUPPORTED_BACKEND(String backend) { 17 | return String.format("Unsupported backend in topic Uri: %s", backend); 18 | } 19 | 20 | public static String NO_SUBSCRIPTION_ASSIGNMENT(String api) { 21 | return String.format("PSC consumer has no existing subscription or assignment before a call to %s", api); 22 | } 23 | 24 | public static String MUTUALLY_EXCLUSIVE_APIS(String calledApi, String usedApi) { 25 | return String.format("%s is not supported when %s is being used.", calledApi, usedApi); 26 | } 27 | 28 | public static String DUPLICATE_PARTITIONS_IN_MESSAGE_IDS(Set topicUriPartitions) { 29 | return String.format("The set of message ids contains duplicate message ids for partition(s): %s", 30 | String.join( 31 | ", ", 32 | topicUriPartitions.stream().map(TopicUriPartition::toString).collect(Collectors.toSet()) 33 | ) 34 | ); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/PscException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception; 2 | 3 | public class PscException extends Exception { 4 | 5 | private static final long serialVersionUID = 1L; 6 | 7 | public PscException(String message) { 8 | super(message); 9 | } 10 | 11 | public PscException(String message, Throwable cause) { 12 | super(message, cause); 13 | } 14 | 15 | public PscException(Throwable cause) { 16 | super(cause); 17 | } 18 | 19 | public String getMetricName() { 20 | return "error." + this.getClass().getName(); 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/consumer/BackendConsumerException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.consumer; 2 | 3 | import com.pinterest.psc.exception.BackendException; 4 | 5 | public class BackendConsumerException extends ConsumerException implements BackendException { 6 | 7 | private static final long serialVersionUID = 1L; 8 | private static final String ERROR_METRIC_PREFIX = "error.consumer."; 9 | 10 | private final String backend; 11 | 12 | public BackendConsumerException(String message, Throwable cause, String backend) { 13 | super(message, cause); 14 | this.backend = backend; 15 | } 16 | 17 | public BackendConsumerException(Throwable cause, String backend) { 18 | super(cause); 19 | this.backend = backend; 20 | 21 | } 22 | 23 | public BackendConsumerException(String message, String backend) { 24 | super(message); 25 | this.backend = backend; 26 | } 27 | 28 | @Override 29 | public String getBackend() { 30 | return this.backend; 31 | } 32 | 33 | @Override 34 | public String getMetricName() { 35 | return ERROR_METRIC_PREFIX + this.backend + "." + 36 | (this.getCause() == null ? this.getClass().getName() : this.getCause().getClass().getName()); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/consumer/ConsumerException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.consumer; 2 | 3 | import com.pinterest.psc.exception.ClientException; 4 | 5 | /** 6 | * Most errors that are thrown when processing {@link com.pinterest.psc.consumer.PscConsumer} APIs are wrapped in an 7 | * exception of type ConsumerException. 8 | */ 9 | public class ConsumerException extends ClientException { 10 | private static final long serialVersionUID = 1L; 11 | 12 | public ConsumerException(String message) { 13 | super(message); 14 | } 15 | 16 | public ConsumerException(String message, Throwable cause) { 17 | super(message, cause); 18 | } 19 | 20 | public ConsumerException(Throwable cause) { 21 | super(cause); 22 | } 23 | } 24 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/consumer/DeserializerException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.consumer; 2 | 3 | /** 4 | * This exception is thrown if the consumed message key or value cannot be deserialized using the configured key or 5 | * value deserializer. This normally means a mismatch between the type of stored data and the expected type by 6 | * {@link com.pinterest.psc.consumer.PscConsumer}. 7 | */ 8 | public class DeserializerException extends ConsumerException { 9 | private static final long serialVersionUID = 1L; 10 | 11 | public DeserializerException(String message) { 12 | super(message); 13 | } 14 | 15 | public DeserializerException(String message, Throwable cause) { 16 | super(message, cause); 17 | } 18 | 19 | public DeserializerException(Throwable cause) { 20 | super(cause); 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/consumer/KeyDeserializerException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.consumer; 2 | 3 | /** 4 | * This exception is thrown if the consumed message key cannot be deserialized using the configured key deserializer. 5 | * This normally means a mismatch between the type of stored data and the expected type by 6 | * {@link com.pinterest.psc.consumer.PscConsumer}. 7 | */ 8 | public class KeyDeserializerException extends DeserializerException { 9 | private static final long serialVersionUID = 1L; 10 | 11 | public KeyDeserializerException(Throwable cause) { 12 | super(cause); 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/consumer/ValueDeserializerException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.consumer; 2 | 3 | /** 4 | * This exception is thrown if the consumed message value cannot be deserialized using the configured value 5 | * deserializer. This normally means a mismatch between the type of stored data and the expected type by 6 | * {@link com.pinterest.psc.consumer.PscConsumer}. 7 | */ 8 | public class ValueDeserializerException extends DeserializerException { 9 | private static final long serialVersionUID = 1L; 10 | 11 | public ValueDeserializerException(Throwable cause) { 12 | super(cause); 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/consumer/WakeupException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.consumer; 2 | 3 | import com.pinterest.psc.consumer.PscConsumer; 4 | 5 | /** 6 | * This exception is thrown as a result of calling {@link PscConsumer#wakeup()}, for example when the consumer is in 7 | * a long running {@link PscConsumer#poll()} call. 8 | */ 9 | public class WakeupException extends ConsumerException { 10 | private static final long serialVersionUID = 1L; 11 | 12 | public WakeupException() { 13 | super(""); 14 | } 15 | 16 | public WakeupException(String message) { 17 | super(message); 18 | } 19 | 20 | public WakeupException(String message, Throwable cause) { 21 | super(message, cause); 22 | } 23 | 24 | public WakeupException(Throwable cause) { 25 | super(cause); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/producer/BackendProducerException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.producer; 2 | 3 | import com.pinterest.psc.exception.BackendException; 4 | 5 | public class BackendProducerException extends ProducerException implements BackendException { 6 | 7 | private static final long serialVersionUID = 1L; 8 | private static final String ERROR_METRIC_PREFIX = "error.producer."; 9 | 10 | private final String backend; 11 | 12 | public BackendProducerException(String message, Throwable cause, String backend) { 13 | super(message, cause); 14 | this.backend = backend; 15 | } 16 | 17 | public BackendProducerException(Throwable cause, String backend) { 18 | super(cause); 19 | this.backend = backend; 20 | } 21 | 22 | public BackendProducerException(String message, String backend) { 23 | super(message); 24 | this.backend = backend; 25 | } 26 | 27 | @Override 28 | public String getBackend() { 29 | return backend; 30 | } 31 | 32 | @Override 33 | public String getMetricName() { 34 | return ERROR_METRIC_PREFIX + this.backend + "." + 35 | (this.getCause() == null ? this.getClass().getName() : this.getCause().getClass().getName()); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/producer/KeySerializerException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.producer; 2 | 3 | /** 4 | * This exception is thrown if the message key that is being produced cannot be serialized using the configured 5 | * key serializer. 6 | */ 7 | public class KeySerializerException extends SerializerException { 8 | private static final long serialVersionUID = 1L; 9 | 10 | public KeySerializerException(Throwable cause) { 11 | super(cause); 12 | } 13 | } 14 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/producer/ProducerException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.producer; 2 | 3 | import com.pinterest.psc.exception.ClientException; 4 | 5 | /** 6 | * Most errors that are thrown when processing {@link com.pinterest.psc.producer.PscProducer} APIs are wrapped in an 7 | * exception of type ProducerException. 8 | */ 9 | public class ProducerException extends ClientException { 10 | private static final long serialVersionUID = 1L; 11 | 12 | public ProducerException(String message) { 13 | super(message); 14 | } 15 | 16 | public ProducerException(String message, Throwable cause) { 17 | super(message, cause); 18 | } 19 | 20 | public ProducerException(Throwable cause) { 21 | super(cause); 22 | } 23 | } 24 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/producer/SerializerException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.producer; 2 | 3 | /** 4 | * This exception is thrown if the message key or value that is being produced cannot be serialized using the configured 5 | * key or value serializer. 6 | */ 7 | public class SerializerException extends ProducerException { 8 | private static final long serialVersionUID = 1L; 9 | 10 | public SerializerException(String message) { 11 | super(message); 12 | } 13 | 14 | public SerializerException(String message, Throwable cause) { 15 | super(message, cause); 16 | } 17 | 18 | public SerializerException(Throwable cause) { 19 | super(cause); 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/producer/TransactionalProducerException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.producer; 2 | 3 | public class TransactionalProducerException extends ProducerException { 4 | public TransactionalProducerException(String message) { 5 | super(message); 6 | } 7 | 8 | public TransactionalProducerException(String message, Throwable cause) { 9 | super(message, cause); 10 | } 11 | 12 | public TransactionalProducerException(Throwable cause) { 13 | super(cause); 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/producer/ValueSerializerException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.producer; 2 | 3 | /** 4 | * This exception is thrown if the message value that is being produced cannot be serialized using the configured 5 | * value serializer. 6 | */ 7 | public class ValueSerializerException extends SerializerException { 8 | private static final long serialVersionUID = 1L; 9 | 10 | public ValueSerializerException(Throwable cause) { 11 | super(cause); 12 | } 13 | } 14 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/startup/ConfigurationException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.startup; 2 | 3 | /** 4 | * This exception is thrown if there are errors with the provided configuration. 5 | */ 6 | public class ConfigurationException extends PscStartupException { 7 | private static final long serialVersionUID = 1L; 8 | 9 | public ConfigurationException(String message) { 10 | super(message); 11 | } 12 | 13 | public ConfigurationException(String message, Throwable cause) { 14 | super(message, cause); 15 | } 16 | 17 | public ConfigurationException(Throwable cause) { 18 | super(cause); 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/startup/PscStartupException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.startup; 2 | 3 | import com.pinterest.psc.exception.PscException; 4 | 5 | public class PscStartupException extends PscException { 6 | 7 | private static final long serialVersionUID = 1L; 8 | 9 | public PscStartupException(String message) { 10 | super(message); 11 | } 12 | 13 | public PscStartupException(String message, Throwable cause) { 14 | super(message, cause); 15 | } 16 | 17 | public PscStartupException(Throwable cause) { 18 | super(cause); 19 | } 20 | 21 | } 22 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/startup/ServiceDiscoveryException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.startup; 2 | 3 | public class ServiceDiscoveryException extends PscStartupException { 4 | private static final long serialVersionUID = 1L; 5 | 6 | public ServiceDiscoveryException(String message) { 7 | super(message); 8 | } 9 | 10 | public ServiceDiscoveryException(String message, Throwable cause) { 11 | super(message, cause); 12 | } 13 | 14 | public ServiceDiscoveryException(Throwable cause) { 15 | super(cause); 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/startup/TopicRnSyntaxException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.startup; 2 | 3 | public class TopicRnSyntaxException extends PscStartupException { 4 | private static final long serialVersionUID = 1L; 5 | 6 | public TopicRnSyntaxException(String input, String reason) { 7 | super("Topic RN '" + input + "' is invalid due to: " + reason); 8 | } 9 | 10 | public TopicRnSyntaxException(String message) { 11 | super(message); 12 | } 13 | 14 | public TopicRnSyntaxException(String message, Throwable cause) { 15 | super(message, cause); 16 | } 17 | 18 | public TopicRnSyntaxException(Throwable cause) { 19 | super(cause); 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/exception/startup/TopicUriSyntaxException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.exception.startup; 2 | 3 | public class TopicUriSyntaxException extends PscStartupException { 4 | private static final long serialVersionUID = 1L; 5 | 6 | public TopicUriSyntaxException(String input, String reason) { 7 | super("Topic URI '" + input + "' is invalid due to: " + reason); 8 | } 9 | 10 | public TopicUriSyntaxException(String input, String reason, Throwable cause) { 11 | super("Topic URI '" + input + "' is invalid due to: " + reason, cause); 12 | } 13 | 14 | public TopicUriSyntaxException(String message) { 15 | super(message); 16 | } 17 | 18 | public TopicUriSyntaxException(String message, Throwable cause) { 19 | super(message, cause); 20 | } 21 | 22 | public TopicUriSyntaxException(Throwable cause) { 23 | super(cause); 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/interceptor/ConsumerInterceptor.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.interceptor; 2 | 3 | import com.pinterest.psc.common.MessageId; 4 | import com.pinterest.psc.consumer.PscConsumerMessage; 5 | import com.pinterest.psc.interceptor.Interceptor; 6 | import org.apache.kafka.common.annotation.InterfaceStability; 7 | 8 | import java.util.Collection; 9 | 10 | public interface ConsumerInterceptor extends Interceptor { 11 | /** 12 | * This interceptor is invoked after consumption of each message and its conversion to a PSC message format. 13 | * The message will run through all default and configured raw interceptors, deserializer interceptor, and types 14 | * interceptors one by one. 15 | * 16 | * @param message The PscConsumer message that is created from the backend consumer message to return to client. 17 | * @return A PSC consumer message that, depending on the interceptor type, may or may not be the same as the input 18 | * message. 19 | */ 20 | PscConsumerMessage onConsume(PscConsumerMessage message); 21 | 22 | /** 23 | * This interceptor is invoked upon consumer commit call where specific message ids are involved. Note that, 24 | * at this time, consumer calls to commitSync() and commitAsync() (with no argument) do 25 | * not trigger this interceptor. 26 | * 27 | * @param messageIds The message is whose offset is being committed. 28 | */ 29 | @InterfaceStability.Evolving 30 | default void onCommit(Collection messageIds) { 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/interceptor/FirstOffsetInterceptor.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.interceptor; 2 | 3 | import com.pinterest.psc.common.TopicUriPartition; 4 | import com.pinterest.psc.consumer.PscConsumerMessage; 5 | import com.pinterest.psc.logging.PscLogger; 6 | 7 | import java.util.HashSet; 8 | import java.util.Set; 9 | 10 | public class FirstOffsetInterceptor extends TypePreservingInterceptor { 11 | private static final PscLogger logger = PscLogger.getLogger(FirstOffsetInterceptor.class); 12 | private Set seenPartitions = new HashSet<>(); 13 | 14 | @Override 15 | public PscConsumerMessage onConsume(PscConsumerMessage message) { 16 | TopicUriPartition topicUriPartition = message.getMessageId().getTopicUriPartition(); 17 | if (!seenPartitions.contains(topicUriPartition)) { 18 | logger.info("First consumed offset of {}: {}", topicUriPartition, message.getMessageId().getOffset()); 19 | seenPartitions.add(topicUriPartition); 20 | } 21 | return super.onConsume(message); 22 | } 23 | } 24 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/interceptor/Interceptor.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.interceptor; 2 | 3 | import com.pinterest.psc.common.PscPlugin; 4 | 5 | public interface Interceptor extends PscPlugin { 6 | } 7 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/interceptor/Interceptors.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.interceptor; 2 | 3 | import com.pinterest.psc.config.PscConfigurationInternal; 4 | 5 | import java.util.List; 6 | 7 | public class Interceptors { 8 | List> rawDataInterceptors; 9 | List> typedDataInterceptors; 10 | PscConfigurationInternal pscConfigurationInternal; 11 | 12 | public Interceptors( 13 | List> rawDataInterceptors, 14 | List> typedDataInterceptors, 15 | PscConfigurationInternal pscConfigurationInternal 16 | ) { 17 | if (rawDataInterceptors != null) 18 | rawDataInterceptors.forEach(interceptor -> interceptor.setPscConfigurationInternal(pscConfigurationInternal)); 19 | if (typedDataInterceptors != null) 20 | typedDataInterceptors.forEach(interceptor -> interceptor.setPscConfigurationInternal(pscConfigurationInternal)); 21 | this.rawDataInterceptors = rawDataInterceptors; 22 | this.typedDataInterceptors = typedDataInterceptors; 23 | this.pscConfigurationInternal = pscConfigurationInternal; 24 | } 25 | 26 | public void addRawDataInterceptor(TypePreservingInterceptor interceptor) { 27 | interceptor.setPscConfigurationInternal(pscConfigurationInternal); 28 | rawDataInterceptors.add(interceptor); 29 | } 30 | 31 | public void addTypedDataInterceptor(TypePreservingInterceptor interceptor) { 32 | interceptor.setPscConfigurationInternal(pscConfigurationInternal); 33 | typedDataInterceptors.add(interceptor); 34 | } 35 | 36 | public List> getRawDataInterceptors() { 37 | return rawDataInterceptors; 38 | } 39 | 40 | public List> getTypedDataInterceptors() { 41 | return typedDataInterceptors; 42 | } 43 | 44 | public PscConfigurationInternal getPscConfigurationInternal() { 45 | return pscConfigurationInternal; 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/interceptor/ProducerInterceptor.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.interceptor; 2 | 3 | import com.pinterest.psc.interceptor.Interceptor; 4 | import com.pinterest.psc.producer.PscProducerMessage; 5 | 6 | public interface ProducerInterceptor extends Interceptor { 7 | /** 8 | * This interceptor is invoked upon producer send calls on each PscProducer message. The message will run through 9 | * all default and configured typed interceptors, serializer interceptor, and raw interceptors one by one. 10 | * 11 | * @param message The PscProducer message or a transformed version of it as it goes through the interceptors. 12 | * @return A PSC producer message that, depending on the interceptor type, may or may not be the same as the input 13 | * message. 14 | */ 15 | PscProducerMessage onSend(PscProducerMessage message); 16 | } 17 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/interceptor/TypePreservingInterceptor.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.interceptor; 2 | 3 | import com.google.common.annotations.VisibleForTesting; 4 | import com.pinterest.psc.common.MessageId; 5 | import com.pinterest.psc.config.PscConfigurationInternal; 6 | import com.pinterest.psc.consumer.PscConsumerMessage; 7 | import com.pinterest.psc.producer.PscProducerMessage; 8 | 9 | import java.util.Collection; 10 | 11 | public abstract class TypePreservingInterceptor 12 | implements ProducerInterceptor, ConsumerInterceptor { 13 | 14 | protected PscConfigurationInternal pscConfigurationInternal; 15 | 16 | @Override 17 | public PscConsumerMessage onConsume(PscConsumerMessage message) { 18 | return message; 19 | } 20 | 21 | @Override 22 | public PscProducerMessage onSend(PscProducerMessage message) { 23 | return message; 24 | } 25 | 26 | @Override 27 | public void onCommit(Collection messageIds) { 28 | } 29 | 30 | @VisibleForTesting 31 | protected TypePreservingInterceptor setPscConfigurationInternal(PscConfigurationInternal pscConfigurationInternal) { 32 | if (pscConfigurationInternal != null) 33 | this.pscConfigurationInternal = pscConfigurationInternal; 34 | return this; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/metrics/Metric.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.metrics; 2 | 3 | public class Metric { 4 | private final MetricName metricName; 5 | private final MetricValueProvider metricValueProvider; 6 | 7 | public Metric(MetricName metricName, MetricValueProvider metricValueProvider) { 8 | this.metricName = metricName; 9 | this.metricValueProvider = metricValueProvider; 10 | } 11 | 12 | public MetricName metricName() { 13 | return metricName; 14 | } 15 | 16 | public Object metricValue() { 17 | return metricValueProvider.getValue(metricName); 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/metrics/MetricName.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.metrics; 2 | 3 | import com.pinterest.psc.common.PscCommon; 4 | 5 | import java.util.Map; 6 | 7 | public final class MetricName { 8 | private final String name; 9 | private final String group; 10 | private final String description; 11 | private Map tags; 12 | private int hash = 0; 13 | 14 | public MetricName(String name, String group, String description, Map tags) { 15 | this.name = PscCommon.verifyNotNull(name); 16 | this.group = PscCommon.verifyNotNull(group); 17 | this.description = PscCommon.verifyNotNull(description); 18 | this.tags = PscCommon.verifyNotNull(tags); 19 | } 20 | 21 | public String name() { 22 | return this.name; 23 | } 24 | 25 | public String group() { 26 | return this.group; 27 | } 28 | 29 | public Map tags() { 30 | return this.tags; 31 | } 32 | 33 | public String description() { 34 | return this.description; 35 | } 36 | 37 | @Override 38 | public int hashCode() { 39 | if (hash != 0) 40 | return hash; 41 | final int prime = 31; 42 | int result = 1; 43 | result = prime * result + group.hashCode(); 44 | result = prime * result + name.hashCode(); 45 | result = prime * result + tags.hashCode(); 46 | this.hash = result; 47 | return result; 48 | } 49 | 50 | @Override 51 | public boolean equals(Object obj) { 52 | if (this == obj) 53 | return true; 54 | if (obj == null) 55 | return false; 56 | if (getClass() != obj.getClass()) 57 | return false; 58 | MetricName other = (MetricName) obj; 59 | return group.equals(other.group) && name.equals(other.name) && tags.equals(other.tags); 60 | } 61 | 62 | @Override 63 | public String toString() { 64 | return String.format( 65 | "MetricName [name=%s, group=%s, description=%s, tags=%s]", 66 | name, group, description, tags 67 | ); 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/metrics/MetricValueProvider.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.metrics; 2 | 3 | import java.util.HashMap; 4 | import java.util.Map; 5 | import java.util.stream.Collectors; 6 | 7 | public class MetricValueProvider { 8 | private final Map internalMetrics = new HashMap<>(); 9 | private final Map externalMetrics = new HashMap<>(); 10 | 11 | public MetricValueProvider() { 12 | } 13 | 14 | public void updateMetrics(Map metrics) { 15 | reset(); 16 | internalMetrics.putAll(metrics); 17 | externalMetrics.putAll(metrics.entrySet().stream().collect(Collectors.toMap( 18 | Map.Entry::getKey, 19 | entry -> new Metric(entry.getKey(), this) 20 | ))); 21 | } 22 | 23 | public void reset() { 24 | internalMetrics.clear(); 25 | externalMetrics.clear(); 26 | } 27 | 28 | public void updateMetric(MetricName metricName, Object metric) { 29 | internalMetrics.put(metricName, metric); 30 | if (!externalMetrics.containsKey(metricName)) 31 | externalMetrics.put(metricName, new Metric(metricName, this)); 32 | } 33 | 34 | public Object getValue(MetricName metricName) { 35 | return internalMetrics.get(metricName); 36 | } 37 | 38 | public Map getMetrics() { 39 | return externalMetrics; 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/metrics/NullMetricsReporter.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.metrics; 2 | 3 | import com.codahale.metrics.Counter; 4 | import com.codahale.metrics.Gauge; 5 | import com.codahale.metrics.Histogram; 6 | import com.codahale.metrics.Meter; 7 | import com.codahale.metrics.MetricFilter; 8 | import com.codahale.metrics.MetricRegistry; 9 | import com.codahale.metrics.Timer; 10 | import com.pinterest.psc.config.MetricsReporterConfiguration; 11 | 12 | import java.util.SortedMap; 13 | import java.util.concurrent.TimeUnit; 14 | 15 | public class NullMetricsReporter extends MetricsReporter { 16 | 17 | protected NullMetricsReporter( 18 | String baseName, 19 | PscMetricTag pscMetricTag, 20 | MetricRegistry registry, 21 | MetricFilter filter, 22 | TimeUnit rateUnit, 23 | TimeUnit durationUnit) { 24 | super(registry, pscMetricTag.getId(), filter, rateUnit, durationUnit); 25 | } 26 | 27 | public static NullMetricsReporter createReporter(MetricsReporterConfiguration metricsReporterConfiguration, 28 | String baseName, 29 | PscMetricTag pscMetricTag, 30 | MetricRegistry registry, 31 | MetricFilter filter, 32 | TimeUnit rateUnit, 33 | TimeUnit durationUnit) { 34 | return new NullMetricsReporter(baseName, pscMetricTag, registry, filter, rateUnit, durationUnit); 35 | } 36 | 37 | @Override 38 | public void report(@SuppressWarnings("rawtypes") SortedMap gauges, 39 | SortedMap counters, 40 | SortedMap histograms, 41 | SortedMap meters, 42 | SortedMap timers) { 43 | } 44 | 45 | } -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/metrics/PscMetricType.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.metrics; 2 | 3 | public enum PscMetricType { 4 | HISTOGRAM, COUNTER 5 | } 6 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/metrics/kafka/KafkaUtils.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.metrics.kafka; 2 | 3 | import com.pinterest.psc.common.PscUtils; 4 | import com.pinterest.psc.metrics.MetricValueProvider; 5 | import org.apache.kafka.common.Metric; 6 | import org.apache.kafka.common.MetricName; 7 | 8 | import java.util.Map; 9 | 10 | public class KafkaUtils { 11 | public static void convertKafkaMetricsToPscMetrics( 12 | Map kafkaMetrics, 13 | MetricValueProvider metricValueProvider 14 | ) { 15 | if (kafkaMetrics == null) 16 | return; 17 | 18 | metricValueProvider.reset(); 19 | kafkaMetrics.forEach((key, value) -> { 20 | Map tags = key.tags(); 21 | tags.put("backend", PscUtils.BACKEND_TYPE_KAFKA); 22 | com.pinterest.psc.metrics.MetricName metricName = new com.pinterest.psc.metrics.MetricName( 23 | key.name(), key.group(), key.description(), tags 24 | ); 25 | metricValueProvider.updateMetric(metricName, value.metricValue()); 26 | }); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/producer/Callback.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.producer; 2 | 3 | import com.pinterest.psc.common.MessageId; 4 | 5 | public interface Callback { 6 | void onCompletion(MessageId messageId, Exception exception); 7 | } 8 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/producer/PscProducerTransactionalProperties.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.producer; 2 | 3 | public class PscProducerTransactionalProperties { 4 | 5 | protected long producerId; 6 | protected short epoch; 7 | 8 | public PscProducerTransactionalProperties(long producerId, short epoch) { 9 | this.producerId = producerId; 10 | this.epoch = epoch; 11 | } 12 | 13 | public long getProducerId() { 14 | return producerId; 15 | } 16 | 17 | public short getEpoch() { 18 | return epoch; 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/producer/creation/PscProducerCreatorPlugin.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.producer.creation; 2 | 3 | import java.lang.annotation.ElementType; 4 | import java.lang.annotation.Retention; 5 | import java.lang.annotation.RetentionPolicy; 6 | import java.lang.annotation.Target; 7 | 8 | @Retention(RetentionPolicy.RUNTIME) 9 | @Target({ElementType.TYPE}) 10 | public @interface PscProducerCreatorPlugin { 11 | String backend(); 12 | } -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/producer/kafka/KafkaProducerTransactionalProperties.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.producer.kafka; 2 | 3 | import com.pinterest.psc.producer.PscProducerTransactionalProperties; 4 | 5 | /** 6 | * Moved to PscProducerTransactionalProperties 7 | */ 8 | @Deprecated 9 | public class KafkaProducerTransactionalProperties extends PscProducerTransactionalProperties { 10 | 11 | public KafkaProducerTransactionalProperties(long producerId, short epoch) { 12 | super(producerId, epoch); 13 | } 14 | 15 | public long getProducerId() { 16 | return producerId; 17 | } 18 | 19 | public short getEpoch() { 20 | return epoch; 21 | } 22 | 23 | @Override 24 | public String toString() { 25 | return "KafkaProducerTransactionalProperties {" + 26 | "producerId=" + producerId + 27 | ", epoch=" + epoch + 28 | '}'; 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/ByteArrayDeserializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | public class ByteArrayDeserializer implements Deserializer { 4 | 5 | @Override 6 | public byte[] deserialize(byte[] bytes) { 7 | return bytes; 8 | } 9 | } 10 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/ByteArraySerializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | public class ByteArraySerializer implements Serializer { 4 | 5 | @Override 6 | public byte[] serialize(byte[] data) { 7 | return data; 8 | } 9 | } 10 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/ByteBufferDeserializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | import java.nio.ByteBuffer; 4 | 5 | public class ByteBufferDeserializer implements Deserializer { 6 | @Override 7 | public ByteBuffer deserialize(byte[] bytes) { 8 | if (bytes == null) 9 | return null; 10 | 11 | return ByteBuffer.wrap(bytes); 12 | } 13 | } 14 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/ByteBufferSerializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | import java.nio.ByteBuffer; 4 | 5 | public class ByteBufferSerializer implements Serializer { 6 | @Override 7 | public byte[] serialize(ByteBuffer data) { 8 | if (data == null) 9 | return null; 10 | 11 | data.rewind(); 12 | 13 | if (data.hasArray()) { 14 | byte[] arr = data.array(); 15 | if (data.arrayOffset() == 0 && arr.length == data.remaining()) { 16 | return arr; 17 | } 18 | } 19 | 20 | byte[] ret = new byte[data.remaining()]; 21 | data.get(ret, 0, ret.length); 22 | data.rewind(); 23 | return ret; 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/Deserializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | import com.pinterest.psc.common.PscPlugin; 4 | import com.pinterest.psc.config.PscConfiguration; 5 | import com.pinterest.psc.exception.consumer.DeserializerException; 6 | 7 | import java.io.Closeable; 8 | 9 | public interface Deserializer extends PscPlugin, Closeable { 10 | default void configure(PscConfiguration pscConfiguration, boolean isKey) { 11 | } 12 | 13 | default void close() { 14 | } 15 | 16 | T deserialize(byte[] bytes) throws DeserializerException; 17 | } 18 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/DoubleDeserializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | import com.pinterest.psc.exception.consumer.DeserializerException; 4 | 5 | public class DoubleDeserializer implements Deserializer { 6 | @Override 7 | public Double deserialize(byte[] bytes) throws DeserializerException { 8 | if (bytes == null) 9 | return null; 10 | if (bytes.length != 8) { 11 | throw new DeserializerException("Size of data received by Deserializer is not 8"); 12 | } 13 | 14 | long value = 0; 15 | for (byte b : bytes) { 16 | value <<= 8; 17 | value |= b & 0xFF; 18 | } 19 | return Double.longBitsToDouble(value); 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/DoubleSerializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | public class DoubleSerializer implements Serializer { 4 | @Override 5 | public byte[] serialize(Double data) { 6 | if (data == null) 7 | return null; 8 | 9 | long bits = Double.doubleToLongBits(data); 10 | return new byte[]{ 11 | (byte) (bits >>> 56), 12 | (byte) (bits >>> 48), 13 | (byte) (bits >>> 40), 14 | (byte) (bits >>> 32), 15 | (byte) (bits >>> 24), 16 | (byte) (bits >>> 16), 17 | (byte) (bits >>> 8), 18 | (byte) bits 19 | }; 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/FloatDeserializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | import com.pinterest.psc.exception.consumer.DeserializerException; 4 | 5 | public class FloatDeserializer implements Deserializer { 6 | @Override 7 | public Float deserialize(byte[] bytes) throws DeserializerException { 8 | if (bytes == null) 9 | return null; 10 | if (bytes.length != 4) { 11 | throw new DeserializerException("Size of data received by Deserializer is not 4"); 12 | } 13 | 14 | int value = 0; 15 | for (byte b : bytes) { 16 | value <<= 8; 17 | value |= b & 0xFF; 18 | } 19 | return Float.intBitsToFloat(value); 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/FloatSerializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | public class FloatSerializer implements Serializer { 4 | @Override 5 | public byte[] serialize(Float data) { 6 | if (data == null) 7 | return null; 8 | 9 | long bits = Float.floatToRawIntBits(data); 10 | return new byte[]{ 11 | (byte) (bits >>> 24), 12 | (byte) (bits >>> 16), 13 | (byte) (bits >>> 8), 14 | (byte) bits 15 | }; 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/IntegerDeserializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | import com.pinterest.psc.exception.consumer.DeserializerException; 4 | 5 | public class IntegerDeserializer implements Deserializer { 6 | 7 | @Override 8 | public Integer deserialize(byte[] bytes) throws DeserializerException { 9 | if (bytes == null) 10 | return null; 11 | if (bytes.length != 4) { 12 | throw new DeserializerException("Size of data received by IntegerDeserializer is not 4"); 13 | } 14 | 15 | int value = 0; 16 | for (byte b : bytes) { 17 | value <<= 8; 18 | value |= b & 0xFF; 19 | } 20 | return value; 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/IntegerSerializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | public class IntegerSerializer implements Serializer { 4 | @Override 5 | public byte[] serialize(Integer data) { 6 | if (data == null) 7 | return null; 8 | 9 | return new byte[]{ 10 | (byte) (data >>> 24), 11 | (byte) (data >>> 16), 12 | (byte) (data >>> 8), 13 | data.byteValue() 14 | }; 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/LongDeserializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | import com.pinterest.psc.exception.consumer.DeserializerException; 4 | 5 | public class LongDeserializer implements Deserializer { 6 | @Override 7 | public Long deserialize(byte[] bytes) throws DeserializerException { 8 | if (bytes == null) 9 | return null; 10 | if (bytes.length != 8) { 11 | throw new DeserializerException("Size of data received by LongDeserializer is not 8"); 12 | } 13 | 14 | long value = 0; 15 | for (byte b : bytes) { 16 | value <<= 8; 17 | value |= b & 0xFF; 18 | } 19 | return value; 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/LongSerializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | public class LongSerializer implements Serializer { 4 | @Override 5 | public byte[] serialize(Long data) { 6 | if (data == null) 7 | return null; 8 | 9 | return new byte[]{ 10 | (byte) (data >>> 56), 11 | (byte) (data >>> 48), 12 | (byte) (data >>> 40), 13 | (byte) (data >>> 32), 14 | (byte) (data >>> 24), 15 | (byte) (data >>> 16), 16 | (byte) (data >>> 8), 17 | data.byteValue() 18 | }; 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/Serializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | import com.pinterest.psc.common.PscPlugin; 4 | import com.pinterest.psc.config.PscConfiguration; 5 | import com.pinterest.psc.exception.producer.SerializerException; 6 | 7 | import java.io.Closeable; 8 | 9 | public interface Serializer extends PscPlugin, Closeable { 10 | default void configure(PscConfiguration pscConfiguration, boolean isKey) { 11 | } 12 | 13 | default void close() { 14 | } 15 | 16 | byte[] serialize(T data) throws SerializerException; 17 | } 18 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/ShortDeserializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | import com.pinterest.psc.exception.consumer.DeserializerException; 4 | 5 | public class ShortDeserializer implements Deserializer { 6 | @Override 7 | public Short deserialize(byte[] bytes) throws DeserializerException { 8 | if (bytes == null) 9 | return null; 10 | if (bytes.length != 2) { 11 | throw new DeserializerException("Size of data received by ShortDeserializer is not 2"); 12 | } 13 | 14 | short value = 0; 15 | for (byte b : bytes) { 16 | value <<= 8; 17 | value |= b & 0xFF; 18 | } 19 | return value; 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/ShortSerializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | public class ShortSerializer implements Serializer { 4 | @Override 5 | public byte[] serialize(Short data) { 6 | if (data == null) 7 | return null; 8 | 9 | return new byte[]{ 10 | (byte) (data >>> 8), 11 | data.byteValue() 12 | }; 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/StringDeserializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | import com.pinterest.psc.config.PscConfiguration; 4 | import com.pinterest.psc.exception.consumer.DeserializerException; 5 | 6 | import java.io.UnsupportedEncodingException; 7 | 8 | public class StringDeserializer implements Deserializer { 9 | private String encoding = "UTF8"; 10 | 11 | @Override 12 | public void configure(PscConfiguration pscConfiguration, boolean isKey) { 13 | String configName = isKey ? "deserializer.key.encoding" : "deserializer.value.encoding"; 14 | String encodingValue = pscConfiguration.getString(configName); 15 | if (encodingValue == null) 16 | encodingValue = pscConfiguration.getString("deserializer.encoding"); 17 | if (encodingValue != null) 18 | encoding = encodingValue; 19 | } 20 | 21 | @Override 22 | public String deserialize(byte[] bytes) throws DeserializerException { 23 | try { 24 | if (bytes == null) 25 | return null; 26 | else 27 | return new String(bytes, encoding); 28 | } catch (UnsupportedEncodingException e) { 29 | throw new DeserializerException("Error when deserializing byte[] to string due to unsupported encoding " + encoding); 30 | } 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/StringSerializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | import com.pinterest.psc.config.PscConfiguration; 4 | import com.pinterest.psc.exception.producer.SerializerException; 5 | 6 | import java.io.UnsupportedEncodingException; 7 | 8 | public class StringSerializer implements Serializer { 9 | private String encoding = "UTF8"; 10 | 11 | @Override 12 | public void configure(PscConfiguration pscConfiguration, boolean isKey) { 13 | String configName = isKey ? "serializer.key.encoding" : "serializer.value.encoding"; 14 | String encodingValue = pscConfiguration.getString(configName); 15 | if (encodingValue == null) 16 | encodingValue = pscConfiguration.getString("serializer.encoding"); 17 | if (encodingValue != null) 18 | encoding = encodingValue; 19 | } 20 | 21 | @Override 22 | public byte[] serialize(String data) throws SerializerException { 23 | try { 24 | if (data == null) 25 | return null; 26 | else 27 | return data.getBytes(encoding); 28 | } catch (UnsupportedEncodingException e) { 29 | throw new SerializerException("Error when serializing string to byte[] due to unsupported encoding " + encoding); 30 | } 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/UuidDeserializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | import com.pinterest.psc.config.PscConfiguration; 4 | import com.pinterest.psc.exception.consumer.DeserializerException; 5 | 6 | import java.io.UnsupportedEncodingException; 7 | import java.util.UUID; 8 | 9 | public class UuidDeserializer implements Deserializer { 10 | private String encoding = "UTF8"; 11 | 12 | @Override 13 | public void configure(PscConfiguration pscConfiguration, boolean isKey) { 14 | String configName = isKey ? "deserializer.key.encoding" : "deserializer.value.encoding"; 15 | String encodingValue = pscConfiguration.getString(configName); 16 | if (encodingValue == null) 17 | encodingValue = pscConfiguration.getString("deserializer.encoding"); 18 | if (encodingValue != null) 19 | encoding = encodingValue; 20 | } 21 | 22 | @Override 23 | public UUID deserialize(byte[] bytes) throws DeserializerException { 24 | try { 25 | if (bytes == null) 26 | return null; 27 | else 28 | return UUID.fromString(new String(bytes, encoding)); 29 | } catch (UnsupportedEncodingException e) { 30 | throw new DeserializerException("Error when deserializing byte[] to UUID due to unsupported encoding " + encoding, e); 31 | } catch (IllegalArgumentException e) { 32 | throw new DeserializerException("Error parsing data into UUID", e); 33 | } 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /psc/src/main/java/com/pinterest/psc/serde/UuidSerializer.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.serde; 2 | 3 | import com.pinterest.psc.config.PscConfiguration; 4 | import com.pinterest.psc.exception.producer.SerializerException; 5 | 6 | import java.io.UnsupportedEncodingException; 7 | import java.util.UUID; 8 | 9 | public class UuidSerializer implements Serializer { 10 | private String encoding = "UTF8"; 11 | 12 | @Override 13 | public void configure(PscConfiguration pscConfiguration, boolean isKey) { 14 | String configName = isKey ? "serializer.key.encoding" : "serializer.value.encoding"; 15 | String encodingValue = pscConfiguration.getString(configName); 16 | if (encodingValue == null) 17 | encodingValue = pscConfiguration.getString("serializer.encoding"); 18 | if (encodingValue != null) 19 | encoding = encodingValue; 20 | } 21 | 22 | @Override 23 | public byte[] serialize(UUID data) throws SerializerException { 24 | try { 25 | if (data == null) 26 | return null; 27 | else 28 | return data.toString().getBytes(encoding); 29 | } catch (UnsupportedEncodingException e) { 30 | throw new SerializerException("Error when serializing UUID to byte[] due to unsupported encoding " + encoding); 31 | } 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /psc/src/main/resources-filtered/psc.properties: -------------------------------------------------------------------------------- 1 | version=${project.version} 2 | psc.additionalRequiredConfigs=psc.project 3 | topic.rn.standard=rn -------------------------------------------------------------------------------- /psc/src/main/resources/log4j.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | 18 | 19 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 36 | 37 | 38 | 39 | 40 | 41 | 42 | 43 | 44 | 45 | 46 | 47 | 48 | 49 | -------------------------------------------------------------------------------- /psc/src/test/java/com/pinterest/psc/common/TestPscUtils.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.common; 2 | 3 | import org.junit.jupiter.api.Test; 4 | 5 | import java.util.Random; 6 | 7 | import static org.junit.jupiter.api.Assertions.assertEquals; 8 | 9 | public class TestPscUtils { 10 | private static final Random random = new Random(); 11 | 12 | @Test 13 | void testIntByteArrayConversions() { 14 | assertEquals(0, PscCommon.byteArrayToInt(PscCommon.intToByteArray(0))); 15 | assertEquals(1, PscCommon.byteArrayToInt(PscCommon.intToByteArray(1))); 16 | assertEquals(-1, PscCommon.byteArrayToInt(PscCommon.intToByteArray(-1))); 17 | assertEquals(Integer.MAX_VALUE, PscCommon.byteArrayToInt(PscCommon.intToByteArray(Integer.MAX_VALUE))); 18 | assertEquals(Integer.MIN_VALUE, PscCommon.byteArrayToInt(PscCommon.intToByteArray(Integer.MIN_VALUE))); 19 | 20 | for (int i = 0; i < 100; ++i) { 21 | int num = random.nextInt(); 22 | assertEquals(num, PscCommon.byteArrayToInt(PscCommon.intToByteArray(num))); 23 | assertEquals(-num, PscCommon.byteArrayToInt(PscCommon.intToByteArray(-num))); 24 | } 25 | } 26 | 27 | @Test 28 | void testLongByteArrayConversions() { 29 | assertEquals(0L, PscCommon.byteArrayToLong(PscCommon.longToByteArray(0L))); 30 | assertEquals(1L, PscCommon.byteArrayToLong(PscCommon.longToByteArray(1L))); 31 | assertEquals(-1L, PscCommon.byteArrayToLong(PscCommon.longToByteArray(-1L))); 32 | assertEquals(Long.MAX_VALUE, PscCommon.byteArrayToLong(PscCommon.longToByteArray(Long.MAX_VALUE))); 33 | assertEquals(Long.MIN_VALUE, PscCommon.byteArrayToLong(PscCommon.longToByteArray(Long.MIN_VALUE))); 34 | 35 | for (int i = 0; i < 100; ++i) { 36 | long num = random.nextLong(); 37 | assertEquals(num, PscCommon.byteArrayToLong(PscCommon.longToByteArray(num))); 38 | assertEquals(-num, PscCommon.byteArrayToLong(PscCommon.longToByteArray(-num))); 39 | } 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /psc/src/test/java/com/pinterest/psc/common/TestTopicRn.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.common; 2 | 3 | import com.pinterest.psc.exception.startup.TopicRnSyntaxException; 4 | import org.junit.jupiter.api.Test; 5 | 6 | import java.io.IOException; 7 | import java.io.InputStream; 8 | import java.util.Properties; 9 | 10 | import static org.junit.jupiter.api.Assertions.assertEquals; 11 | import static org.junit.jupiter.api.Assertions.assertThrows; 12 | import static org.junit.jupiter.api.Assertions.assertTrue; 13 | 14 | class TestTopicRn { 15 | private final String valid_rn = TopicUri.STANDARD + ":kafka:env:aws_us-west-1::kafkacluster01:topic01"; 16 | 17 | @Test 18 | void testTopicRnStandard() throws IOException { 19 | InputStream input = getClass().getClassLoader().getResourceAsStream("psc.properties"); 20 | Properties prop = new Properties(); 21 | 22 | // load a properties file 23 | prop.load(input); 24 | 25 | // get the property value and print it out 26 | assertTrue(prop.containsKey("topic.rn.standard")); 27 | assertEquals(TopicUri.STANDARD, prop.getProperty("topic.rn.standard")); 28 | } 29 | 30 | @Test 31 | void testTopicRn() throws Exception { 32 | // bad cases: 33 | // 1. non matching 34 | assertThrows(TopicRnSyntaxException.class, () -> TopicRn.validate("nonmatching")); 35 | 36 | // 2. fewer parts 37 | assertThrows(TopicRnSyntaxException.class, () -> TopicRn.validate("ab:cd")); 38 | 39 | // 3. more parts 40 | assertThrows(TopicRnSyntaxException.class, () -> TopicRn.validate("ab:cd:ef:gh:ij:kl:mn:op:qr")); 41 | 42 | // 4. empty parts 43 | assertThrows(TopicRnSyntaxException.class, () -> TopicRn.validate("std:par:svc:rgn:act::t")); 44 | 45 | // 5. invalid chars 46 | assertThrows(TopicRnSyntaxException.class, () -> TopicRn.validate("std?:par:svc#:rgn+:act::t")); 47 | 48 | TopicRn.validate(valid_rn); 49 | } 50 | } -------------------------------------------------------------------------------- /psc/src/test/java/com/pinterest/psc/common/TestTopicUri.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.common; 2 | 3 | import com.pinterest.psc.common.kafka.KafkaTopicUri; 4 | import com.pinterest.psc.exception.startup.TopicUriSyntaxException; 5 | 6 | public class TestTopicUri extends KafkaTopicUri { 7 | public TestTopicUri(TopicUri baseTopicUri) { 8 | super(baseTopicUri); 9 | } 10 | 11 | public static TestTopicUri validate(TopicUri baseTopicUri) throws TopicUriSyntaxException { 12 | if (baseTopicUri.getTopicRn() == null) 13 | throw new TopicUriSyntaxException(baseTopicUri.getTopicUriAsString(), "Missing path in Test URI"); 14 | if (!baseTopicUri.getBackend().equals(TestUtils.BACKEND_TYPE_TEST)) 15 | throw new TopicUriSyntaxException(baseTopicUri.getTopicUriAsString(), "Not a Test URI"); 16 | if (baseTopicUri.getProtocol() != null && 17 | !baseTopicUri.getProtocol().equals(PLAINTEXT_PROTOCOL) && 18 | !baseTopicUri.getProtocol().equals(SECURE_PROTOCOL)) { 19 | throw new TopicUriSyntaxException( 20 | baseTopicUri.getTopicUriAsString(), "Invalid protocol in Test URI: '" + 21 | baseTopicUri.getProtocol() + "'. " + "Supported protocols are '" + SECURE_PROTOCOL + 22 | "' and '" + PLAINTEXT_PROTOCOL + "'" 23 | ); 24 | } 25 | return new TestTopicUri(baseTopicUri); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /psc/src/test/java/com/pinterest/psc/common/TestUtils.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.common; 2 | 3 | import com.pinterest.psc.metrics.NullMetricsReporter; 4 | 5 | import java.util.Random; 6 | 7 | public class TestUtils { 8 | public static final String BACKEND_TYPE_TEST = "test"; 9 | public static final String DEFAULT_METRICS_REPORTER = NullMetricsReporter.class.getName(); 10 | protected static final Random random = new Random(); 11 | 12 | public static byte[] getRandomBytes(int size) { 13 | return getRandomBytes(size, 0); 14 | } 15 | 16 | public static byte[] getRandomBytes(int size, double chanceOfNull) { 17 | if (random.nextDouble() < chanceOfNull) 18 | return null; 19 | 20 | byte[] bytes = new byte[random.nextInt(256)]; 21 | random.nextBytes(bytes); 22 | return bytes; 23 | } 24 | 25 | public static String getRandomString(int size) { 26 | return getRandomString(size, 0); 27 | } 28 | 29 | public static String getRandomString(int size, boolean alphabeticalOnly) { 30 | return getRandomString(size, 0, alphabeticalOnly); 31 | } 32 | 33 | public static String getRandomString(int size, double chanceOfNull) { 34 | return getRandomString(size, chanceOfNull, false); 35 | } 36 | 37 | public static String getRandomString(int size, double chanceOfNull, boolean alphabeticalOnly) { 38 | if (random.nextDouble() < chanceOfNull) 39 | return null; 40 | 41 | String alphabet = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz"; 42 | int alphabetLength = alphabet.length(); 43 | 44 | StringBuilder stringBuilder = new StringBuilder(); 45 | for (int i = 0; i < size; ++i) { 46 | stringBuilder.append(alphabeticalOnly ? 47 | alphabet.charAt(random.nextInt(alphabetLength)) : 48 | (char) random.nextInt(256)); 49 | } 50 | 51 | return stringBuilder.toString(); 52 | } 53 | 54 | public static TopicUriPartition getFinalizedTopicUriPartition(TopicUri topicUri, int partition) { 55 | return new TopicUriPartition(topicUri, partition); 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /psc/src/test/java/com/pinterest/psc/consumer/BytesPscConsumerMessageTestUtil.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.consumer; 2 | 3 | import com.pinterest.psc.common.TestUtils; 4 | import com.pinterest.psc.exception.startup.TopicUriSyntaxException; 5 | 6 | import java.util.ArrayList; 7 | import java.util.List; 8 | 9 | public class BytesPscConsumerMessageTestUtil extends PscConsumerMessageTestUtil { 10 | @Override 11 | public List> getRandomPscConsumerMessages(int messageCount) throws 12 | TopicUriSyntaxException { 13 | List> pscConsumerMessageList = new ArrayList<>(); 14 | for (int i = 0; i < messageCount; ++i) { 15 | byte[] key = TestUtils.getRandomBytes(16, 0.5); 16 | byte[] value = TestUtils.getRandomBytes(128, 0.1); 17 | pscConsumerMessageList.add(new PscConsumerMessage() 18 | .setKey(key) 19 | .setValue(value) 20 | .setPublishTimestamp(System.currentTimeMillis()) 21 | .setMessageId(getRandomMessageId(key == null ? -1 : key.length, value == null ? -1 : value.length)) 22 | .addHeaders(getRandomHeaders(6)) 23 | .setTags(getRandomPscMessageTags(0.25)) 24 | ); 25 | } 26 | 27 | return pscConsumerMessageList; 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /psc/src/test/java/com/pinterest/psc/consumer/PscConsumerUtils.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.consumer; 2 | 3 | import com.pinterest.psc.consumer.creation.PscConsumerCreatorManager; 4 | import com.pinterest.psc.consumer.listener.MessageListener; 5 | import com.pinterest.psc.interceptor.Interceptors; 6 | import com.pinterest.psc.metrics.PscMetricRegistryManager; 7 | 8 | public class PscConsumerUtils { 9 | public static MessageListener getMessageListener(PscConsumer pscConsumer) { 10 | return pscConsumer.getListener(); 11 | } 12 | 13 | public static Interceptors getInterceptors(PscConsumer pscConsumer) { 14 | return pscConsumer.getInterceptors(); 15 | } 16 | 17 | public static void setCreatorManager(PscConsumer pscConsumer, PscConsumerCreatorManager creatorManager) { 18 | pscConsumer.setCreatorManager(creatorManager); 19 | } 20 | 21 | public static void setPscMetricRegistryManager(PscConsumer pscConsumer, PscMetricRegistryManager pscMetricRegistryManager) { 22 | pscConsumer.setPscMetricRegistryManager(pscMetricRegistryManager); 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /psc/src/test/java/com/pinterest/psc/consumer/StringPscConsumerMessageTestUtil.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.consumer; 2 | 3 | import com.pinterest.psc.common.TestUtils; 4 | import com.pinterest.psc.exception.startup.TopicUriSyntaxException; 5 | 6 | import java.util.ArrayList; 7 | import java.util.List; 8 | 9 | public class StringPscConsumerMessageTestUtil extends PscConsumerMessageTestUtil { 10 | @Override 11 | public List> getRandomPscConsumerMessages(int messageCount) throws 12 | TopicUriSyntaxException { 13 | List> pscConsumerMessageList = new ArrayList<>(); 14 | for (int i = 0; i < messageCount; ++i) { 15 | pscConsumerMessageList.add(new PscConsumerMessage() 16 | .setKey(TestUtils.getRandomString(16, 0.1)) 17 | .setValue(TestUtils.getRandomString(128, 0.1)) 18 | .setPublishTimestamp(System.currentTimeMillis()) 19 | .setMessageId(getRandomMessageId(random.nextInt(64), random.nextInt(102400))) 20 | .addHeaders(getRandomHeaders(10)) 21 | .setTags(getRandomPscMessageTags(0.25)) 22 | ); 23 | } 24 | 25 | return pscConsumerMessageList; 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /psc/src/test/java/com/pinterest/psc/consumer/listener/MessageCounterListener.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.consumer.listener; 2 | 3 | import com.pinterest.psc.consumer.PscConsumer; 4 | import com.pinterest.psc.consumer.PscConsumerMessage; 5 | 6 | public class MessageCounterListener implements MessageListener { 7 | private int handleCallCounter = 0; 8 | 9 | @Override 10 | public void handle(PscConsumer consumer, PscConsumerMessage message) { 11 | ++handleCallCounter; 12 | } 13 | 14 | public int getHandleCallCounter() { 15 | return handleCallCounter; 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /psc/src/test/java/com/pinterest/psc/discovery/TestFallbackServiceDiscoveryProvider.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.discovery; 2 | 3 | import com.pinterest.psc.common.ServiceDiscoveryConfig; 4 | import com.pinterest.psc.common.TopicUri; 5 | import com.pinterest.psc.environment.Environment; 6 | import com.pinterest.psc.exception.startup.TopicUriSyntaxException; 7 | import org.junit.jupiter.api.Test; 8 | import org.junit.jupiter.api.extension.ExtendWith; 9 | import org.mockito.Mock; 10 | import org.mockito.junit.jupiter.MockitoExtension; 11 | 12 | import java.io.IOException; 13 | 14 | import static org.junit.jupiter.api.Assertions.assertEquals; 15 | import static org.junit.jupiter.api.Assertions.assertNull; 16 | 17 | @ExtendWith(MockitoExtension.class) 18 | public class TestFallbackServiceDiscoveryProvider { 19 | 20 | @Mock 21 | Environment environment; 22 | 23 | private static final String testUri1 = "plaintext:" + TopicUri.SEPARATOR + TopicUri.STANDARD + ":kafka:env:aws_us-west-1::kafkacluster01:"; 24 | private static final String nonexistentUri = "plaintext:" + TopicUri.SEPARATOR + TopicUri.STANDARD + ":kafka:env:aws_us-west-1::kafkacluster03:"; 25 | 26 | @Test 27 | void testGetConfigSimple() throws TopicUriSyntaxException, IOException { 28 | ServiceDiscoveryProvider sdp = new FallbackServiceDiscoveryProvider( 29 | DiscoveryUtil.createTempFallbackFile() 30 | ); 31 | ServiceDiscoveryConfig sdc = sdp.getConfig(environment, TopicUri.validate(testUri1)); 32 | assertEquals("kafkacluster01001:9092,kafkacluster01002:9092", sdc.getConnect()); 33 | assertEquals("PLAINTEXT", sdc.getSecurityProtocol()); 34 | } 35 | 36 | @Test 37 | void testGetConfigNonexistentUri() throws TopicUriSyntaxException, IOException { 38 | ServiceDiscoveryProvider sdp = new FallbackServiceDiscoveryProvider( 39 | DiscoveryUtil.createTempFallbackFile() 40 | ); 41 | ServiceDiscoveryConfig sdc = sdp.getConfig(environment, TopicUri.validate(nonexistentUri)); 42 | assertNull(sdc); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /psc/src/test/java/com/pinterest/psc/interceptor/ExceptionalInterceptor.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.interceptor; 2 | 3 | import com.pinterest.psc.common.MessageId; 4 | import com.pinterest.psc.consumer.PscConsumerMessage; 5 | 6 | import java.util.Collection; 7 | 8 | public class ExceptionalInterceptor extends IdentityInterceptor { 9 | 10 | @Override 11 | public PscConsumerMessage onConsume(PscConsumerMessage message) { 12 | super.onConsume(message); 13 | throw new RuntimeException("exception"); 14 | } 15 | 16 | @Override 17 | public void onCommit(Collection messageIds) { 18 | super.onCommit(messageIds); 19 | throw new RuntimeException("exception"); 20 | } 21 | 22 | /* 23 | @Override 24 | public void close() { 25 | super.close(); 26 | throw new RuntimeException("exception"); 27 | } 28 | */ 29 | } 30 | -------------------------------------------------------------------------------- /psc/src/test/java/com/pinterest/psc/interceptor/ExceptionalMetricsReportingInterceptor.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.interceptor; 2 | 3 | import com.pinterest.psc.common.MessageId; 4 | import com.pinterest.psc.consumer.PscConsumerMessage; 5 | import com.pinterest.psc.exception.consumer.DeserializerException; 6 | import com.pinterest.psc.exception.handler.PscErrorHandler; 7 | 8 | import java.util.Collection; 9 | 10 | public class ExceptionalMetricsReportingInterceptor extends IdentityInterceptor { 11 | 12 | @Override 13 | public PscConsumerMessage onConsume(PscConsumerMessage message) { 14 | try { 15 | super.onConsume(message); 16 | throw new DeserializerException("deser exception"); 17 | } catch (DeserializerException e) { 18 | PscErrorHandler.handle( 19 | e, 20 | message.getMessageId().getTopicUriPartition().getTopicUri(), 21 | true, 22 | pscConfigurationInternal 23 | ); 24 | } 25 | return message; 26 | } 27 | 28 | @Override 29 | public void onCommit(Collection messageIds) { 30 | super.onCommit(messageIds); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /psc/src/test/java/com/pinterest/psc/interceptor/IdentityInterceptor.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.interceptor; 2 | 3 | import com.pinterest.psc.common.MessageId; 4 | import com.pinterest.psc.consumer.PscConsumerMessage; 5 | import com.pinterest.psc.interceptor.TypePreservingInterceptor; 6 | 7 | import java.util.Collection; 8 | 9 | public class IdentityInterceptor extends TypePreservingInterceptor { 10 | public int configureCounter = 0; 11 | public int onConsumeCounter = 0; 12 | public int onCommitCounter = 0; 13 | public int closeCounter = 0; 14 | 15 | /* 16 | @Override 17 | public void close() { 18 | closeCounter++; 19 | } 20 | */ 21 | 22 | @Override 23 | public PscConsumerMessage onConsume(PscConsumerMessage message) { 24 | onConsumeCounter++; 25 | return message; 26 | } 27 | 28 | @Override 29 | public void onCommit(Collection messageIds) { 30 | onCommitCounter++; 31 | } 32 | 33 | /* 34 | @Override 35 | public void configure(PscConsumerConfig config) { 36 | configureCounter++; 37 | } 38 | */ 39 | } 40 | -------------------------------------------------------------------------------- /psc/src/test/java/com/pinterest/psc/interceptor/ModifierInterceptor.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.interceptor; 2 | 3 | import com.pinterest.psc.consumer.PscConsumerMessage; 4 | 5 | public class ModifierInterceptor extends IdentityInterceptor { 6 | @Override 7 | public PscConsumerMessage onConsume(PscConsumerMessage message) { 8 | super.onConsume(message); 9 | return message; 10 | } 11 | } 12 | -------------------------------------------------------------------------------- /psc/src/test/java/com/pinterest/psc/metrics/MetricsUtils.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.metrics; 2 | 3 | import com.pinterest.psc.config.PscConfigurationInternal; 4 | 5 | public class MetricsUtils { 6 | public static void resetMetrics(PscMetricRegistryManager pscMetricRegistryManager) { 7 | pscMetricRegistryManager.cleanup(); 8 | pscMetricRegistryManager.clearCurrentThreadMetricMap(); 9 | } 10 | 11 | public static void shutdownMetrics(PscMetricRegistryManager pscMetricRegistryManager, PscConfigurationInternal pscConfigurationInternal) { 12 | pscMetricRegistryManager.shutdown(pscConfigurationInternal); 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /psc/src/test/java/com/pinterest/psc/producer/PscProducerUtils.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.psc.producer; 2 | 3 | import com.pinterest.psc.interceptor.Interceptors; 4 | import com.pinterest.psc.metrics.PscMetricRegistryManager; 5 | import com.pinterest.psc.producer.creation.PscProducerCreatorManager; 6 | 7 | public class PscProducerUtils { 8 | 9 | public static void setCreatorManager(PscProducer pscProducer, PscProducerCreatorManager creatorManager) { 10 | pscProducer.setCreatorManager(creatorManager); 11 | } 12 | 13 | public static void setPscMetricRegistryManager(PscProducer pscProducer, PscMetricRegistryManager pscMetricRegistryManager) { 14 | pscProducer.setPscMetricRegistryManager(pscMetricRegistryManager); 15 | } 16 | 17 | public static Interceptors getInterceptors(PscProducer pscProducer) { 18 | return pscProducer.getInterceptors(); 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /psc/src/test/resources/kafka.keystore.jks: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc/src/test/resources/kafka.keystore.jks -------------------------------------------------------------------------------- /psc/src/test/resources/kafka.truststore.jks: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/psc/6da931cafdb1f76e4a661e1b40e8742e0d152c7a/psc/src/test/resources/kafka.truststore.jks -------------------------------------------------------------------------------- /psc/src/test/resources/log4j2-test.properties.alt: -------------------------------------------------------------------------------- 1 | name=PscTestPropertiesConfig 2 | appenders=console 3 | 4 | appender.console.type=Console 5 | appender.console.name=stdOut 6 | appender.console.layout.type=PatternLayout 7 | appender.console.layout.pattern=[%d] %-5p %m (%c:%L)%n 8 | 9 | rootLogger.level=info 10 | rootLogger.appenderRefs=stdout 11 | rootLogger.appenderRef.stdout.ref=stdOut 12 | -------------------------------------------------------------------------------- /psc/src/test/resources/log4j2-test.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | 18 | 19 | -------------------------------------------------------------------------------- /psc/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker: -------------------------------------------------------------------------------- 1 | mock-maker-inline --------------------------------------------------------------------------------