├── .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 | 123 456 789
--------------------------------------------------------------------------------
/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