├── .github ├── CODEOWNERS └── workflows │ └── publish_to_maven_central.yaml ├── .gitignore ├── CONTRIBUTORS.md ├── LICENSE ├── README.md ├── deploy ├── common.sh ├── configs │ ├── clustered.yaml │ └── nonclustered.yaml ├── pom.xml ├── run.sh ├── startlocalkafka.sh └── stoplocalkafka.sh ├── docs ├── architecture.md ├── brokerconfig.md ├── deploy.md ├── gettingstarted.md ├── images │ ├── design.png │ ├── producer.png │ └── protocol.png └── opensource.md ├── memq-client-all ├── .gitignore └── pom.xml ├── memq-client ├── .gitignore ├── pom.xml └── src │ ├── main │ ├── java │ │ └── com │ │ │ └── pinterest │ │ │ └── memq │ │ │ ├── client │ │ │ ├── commons │ │ │ │ ├── AuditorUtils.java │ │ │ │ ├── CommonConfigs.java │ │ │ │ ├── CommonUtils.java │ │ │ │ ├── Compression.java │ │ │ │ ├── ConsumerConfigs.java │ │ │ │ ├── DataCorruptionException.java │ │ │ │ ├── Deserializer.java │ │ │ │ ├── MemqCommonClient.java │ │ │ │ ├── MemqLogMessageIterator.java │ │ │ │ ├── MemqMessageHeader.java │ │ │ │ ├── MemqNettyClientSideResponseHandler.java │ │ │ │ ├── MemqWriteResult.java │ │ │ │ ├── ProducerConfigs.java │ │ │ │ ├── ResponseHandler.java │ │ │ │ ├── SimpleMessageId.java │ │ │ │ ├── UnknownCompressionException.java │ │ │ │ ├── audit │ │ │ │ │ ├── Auditor.java │ │ │ │ │ ├── KafkaBackedAuditAnalyzer.java │ │ │ │ │ └── KafkaBackedAuditor.java │ │ │ │ └── serde │ │ │ │ │ ├── ByteArrayDeserializer.java │ │ │ │ │ ├── ByteArraySerializer.java │ │ │ │ │ ├── Serializer.java │ │ │ │ │ ├── StringDeserializer.java │ │ │ │ │ └── ThriftDeserializer.java │ │ │ ├── commons2 │ │ │ │ ├── ClosedException.java │ │ │ │ ├── DataNotFoundException.java │ │ │ │ ├── Endpoint.java │ │ │ │ ├── MemqCommonClient.java │ │ │ │ ├── TopicNotFoundException.java │ │ │ │ ├── TransportPacketIdentifier.java │ │ │ │ ├── network │ │ │ │ │ ├── ClientClosedException.java │ │ │ │ │ ├── ClosedConnectionException.java │ │ │ │ │ ├── NetworkClient.java │ │ │ │ │ ├── ResponseHandler.java │ │ │ │ │ └── netty │ │ │ │ │ │ ├── ClientChannelInitializer.java │ │ │ │ │ │ ├── ConnectionLifecycleHandler.java │ │ │ │ │ │ └── MemqNettyClientSideResponseHandler.java │ │ │ │ └── retry │ │ │ │ │ ├── ExponentialBackoffRetryStrategy.java │ │ │ │ │ ├── FullJitterRetryStrategy.java │ │ │ │ │ ├── RetryStrategy.java │ │ │ │ │ └── UniformRetryStrategy.java │ │ │ ├── consumer │ │ │ │ ├── KafkaNotificationSource.java │ │ │ │ ├── MemqConsumer.java │ │ │ │ ├── Metrics.java │ │ │ │ ├── NoTopicsSubscribedException.java │ │ │ │ ├── NotificationSource.java │ │ │ │ ├── OffsetCommitCallback.java │ │ │ │ ├── tools │ │ │ │ │ └── ConsoleConsumer.java │ │ │ │ └── utils │ │ │ │ │ ├── CheckNotificationTopic.java │ │ │ │ │ └── properties │ │ │ │ │ ├── MemqConsumerBuilder.java │ │ │ │ │ └── PropertiesInitializationException.java │ │ │ ├── producer │ │ │ │ ├── MemqProducer.java │ │ │ │ ├── MemqWriteResult.java │ │ │ │ ├── TaskRequest.java │ │ │ │ ├── http │ │ │ │ │ └── DaemonThreadFactory.java │ │ │ │ └── netty │ │ │ │ │ ├── MemqNettyProducer.java │ │ │ │ │ └── MemqNettyRequest.java │ │ │ └── producer2 │ │ │ │ ├── MemqProducer.java │ │ │ │ ├── RawRecord.java │ │ │ │ ├── Request.java │ │ │ │ └── RequestManager.java │ │ │ └── commons │ │ │ └── storage │ │ │ ├── DelayedDevNullStorageHandler.java │ │ │ ├── DevNullStorageHandler.java │ │ │ ├── NoOpStorageHandler.java │ │ │ ├── ReadBrokerStorageHandler.java │ │ │ ├── StorageHandler.java │ │ │ ├── StorageHandlerName.java │ │ │ ├── StorageHandlerTable.java │ │ │ ├── SysoutStorageHandler.java │ │ │ ├── WriteFailedException.java │ │ │ ├── fs │ │ │ └── FileSystemStorageHandler.java │ │ │ ├── s3 │ │ │ ├── AbstractS3StorageHandler.java │ │ │ ├── CustomS3Async2StorageHandler.java │ │ │ ├── CustomS3AsyncStorageHandler.java │ │ │ ├── KafkaNotificationSink.java │ │ │ ├── MemqS3DNSResolver.java │ │ │ ├── S3Exception.java │ │ │ └── reader │ │ │ │ └── client │ │ │ │ ├── ApacheRequestClient.java │ │ │ │ ├── ReactorNettyRequestClient.java │ │ │ │ └── RequestClient.java │ │ │ └── s3express │ │ │ ├── S3ExpressAsyncStorageHandler.java │ │ │ ├── S3ExpressHelper.java │ │ │ ├── SessionCreds.java │ │ │ ├── SessionTokenManager.java │ │ │ └── keygenerator │ │ │ ├── DateHourKeyGenerator.java │ │ │ └── S3ExpressObjectKeyGenerator.java │ └── resources │ │ └── log4j.properties │ └── test │ └── java │ └── com │ └── pinterest │ └── memq │ ├── client │ ├── commons │ │ ├── TestUtils.java │ │ ├── ZstdTest.java │ │ └── audit │ │ │ └── TestKafkaBackedAuditor.java │ ├── commons2 │ │ ├── MockMemqServer.java │ │ ├── TestMemqCommonClient.java │ │ └── network │ │ │ └── TestNetworkClient.java │ ├── consumer │ │ ├── TestConsumerIntegration.java │ │ ├── TestData.java │ │ └── TestMemqConsumer.java │ ├── producer │ │ ├── TestMemqProducer.java │ │ └── netty │ │ │ └── TestMemqNettyProducer.java │ └── producer2 │ │ ├── TestMemqProducer.java │ │ ├── TestMemqProducerBase.java │ │ └── TestMemqProducerMemory.java │ └── commons │ └── storage │ ├── TestStorageHandler.java │ ├── fs │ └── TestFileSystemStorageHandler.java │ ├── s3 │ ├── TestAbstractS3StorageHandler.java │ └── TestCustomS3Async2OutputHandler.java │ └── s3express │ ├── TestS3ExpressHelper.java │ └── keygenerator │ └── TestDateHourKeyGenerator.java ├── memq-commons ├── .gitignore ├── pom.xml └── src │ ├── main │ ├── java │ │ └── com │ │ │ └── pinterest │ │ │ └── memq │ │ │ ├── commons │ │ │ ├── BatchHeader.java │ │ │ ├── CloseableIterator.java │ │ │ ├── MemqLogMessage.java │ │ │ ├── MessageId.java │ │ │ ├── config │ │ │ │ └── SSLConfig.java │ │ │ ├── mon │ │ │ │ ├── OpenTSDBClient.java │ │ │ │ └── OpenTSDBReporter.java │ │ │ └── protocol │ │ │ │ ├── BatchData.java │ │ │ │ ├── Broker.java │ │ │ │ ├── ClusterMetadataResponse.java │ │ │ │ ├── MemqConstants.java │ │ │ │ ├── Packet.java │ │ │ │ ├── ProtocolUtils.java │ │ │ │ ├── ReadRequestPacket.java │ │ │ │ ├── ReadResponsePacket.java │ │ │ │ ├── RequestPacket.java │ │ │ │ ├── RequestType.java │ │ │ │ ├── ResponseCodes.java │ │ │ │ ├── ResponsePacket.java │ │ │ │ ├── TopicAssignment.java │ │ │ │ ├── TopicConfig.java │ │ │ │ ├── TopicMetadata.java │ │ │ │ ├── TopicMetadataRequestPacket.java │ │ │ │ ├── TopicMetadataResponsePacket.java │ │ │ │ ├── TransportPacket.java │ │ │ │ ├── WriteRequestPacket.java │ │ │ │ └── WriteResponsePacket.java │ │ │ └── core │ │ │ ├── commons │ │ │ ├── MemqProcessingThreadFactory.java │ │ │ ├── Message.java │ │ │ └── MessageBufferInputStream.java │ │ │ └── utils │ │ │ ├── DaemonThreadFactory.java │ │ │ ├── MemqUtils.java │ │ │ └── MiscUtils.java │ └── thrift │ │ └── memq.thrift │ └── test │ └── java │ └── com │ └── pinterest │ └── memq │ └── commons │ └── protocol │ └── TestTopicMetadataPacket.java ├── memq-examples ├── .gitignore ├── pom.xml └── src │ └── main │ └── java │ └── com │ └── pinterest │ └── memq │ └── client │ └── examples │ ├── ExampleMemqConsumer.java │ ├── ExampleMemqConsumerMultiThreaded.java │ └── ExampleMemqProducer.java ├── memq-python-client ├── README.md ├── pyproject.toml ├── requirements.txt ├── run_tests.sh ├── setup.cfg ├── setup.py ├── src │ └── memq │ │ ├── __init__.py │ │ ├── example_batch_consumer.py │ │ ├── inputstream.py │ │ ├── kafkanotificationsource.py │ │ ├── memqbatchconsumer.py │ │ ├── memqconsumer.py │ │ └── memqlogmessageiterator.py ├── tests │ ├── __init__.py │ └── testmemqlogmessageiterator.py └── tox.ini ├── memq.png ├── memq ├── .gitignore ├── arc │ ├── S3Async2OutputHandler.java │ ├── S3AsyncOutputHandler.java │ ├── S3FileAsync2OutputHandler.java │ ├── S3OutputHandler.java │ └── bucketing │ │ ├── Batch.java │ │ ├── BatchOutputTask.java │ │ ├── BucketingTopicProcessor.java │ │ └── TestBucketingTopicProcessor.java ├── pom.xml └── src │ ├── main │ ├── java │ │ └── com │ │ │ └── pinterest │ │ │ └── memq │ │ │ └── core │ │ │ ├── MemqMain.java │ │ │ ├── MemqManager.java │ │ │ ├── clustering │ │ │ ├── BalanceStrategy.java │ │ │ ├── Balancer.java │ │ │ ├── ExpirationPartitionBalanceStrategy.java │ │ │ ├── ExpirationPartitionBalanceStrategyWithAssignmentFreeze.java │ │ │ ├── ExpirationPartitionBalanceStrategyWithErrorHandling.java │ │ │ ├── MemqGovernor.java │ │ │ ├── MetadataPoller.java │ │ │ ├── PartitionBalanceStrategy.java │ │ │ └── TopicAssignmentWatcher.java │ │ │ ├── config │ │ │ ├── AuthorizerConfig.java │ │ │ ├── ClusteringConfig.java │ │ │ ├── EC2EnvironmentProvider.java │ │ │ ├── EnvironmentProvider.java │ │ │ ├── LocalEnvironmentProvider.java │ │ │ ├── MemqConfig.java │ │ │ ├── NettyServerConfig.java │ │ │ └── OpenTsdbConfiguration.java │ │ │ ├── mon │ │ │ ├── MemqMgrHealthCheck.java │ │ │ └── MonitorEndpoint.java │ │ │ ├── processing │ │ │ ├── Ackable.java │ │ │ ├── MapAcker.java │ │ │ ├── ProcessingStatus.java │ │ │ ├── TopicProcessor.java │ │ │ ├── TopicProcessorState.java │ │ │ └── bucketing │ │ │ │ ├── Batch.java │ │ │ │ ├── BatchManager.java │ │ │ │ └── BucketingTopicProcessor.java │ │ │ ├── rpc │ │ │ ├── BrokerTrafficShapingHandler.java │ │ │ ├── MemqNettyServer.java │ │ │ ├── MemqProtocolFeatureMatrix.java │ │ │ ├── MemqRequestDecoder.java │ │ │ ├── MemqResponseEncoder.java │ │ │ ├── PacketSwitchingHandler.java │ │ │ ├── Response.java │ │ │ ├── ServerConnectionLifecycleHandler.java │ │ │ └── WriteResponse.java │ │ │ ├── security │ │ │ └── Authorizer.java │ │ │ ├── tools │ │ │ └── TopicAdmin.java │ │ │ └── utils │ │ │ ├── CoreUtils.java │ │ │ └── CrcProcessor.java │ └── resources │ │ ├── log4j.properties │ │ ├── logback.xml │ │ └── logo.txt │ └── test │ ├── java │ └── com │ │ └── pinterest │ │ └── memq │ │ └── core │ │ ├── TestMemqManager.java │ │ ├── clustering │ │ ├── TestExpirationPartitionBalanceStrategy.java │ │ ├── TestExpirationPartitionBalanceStrategyWithAssignmentFreeze.java │ │ ├── TestMemqGovernor.java │ │ └── TestPartitionBalanceStrategy.java │ │ ├── commons │ │ └── TestMessageBufferInputStream.java │ │ ├── integration │ │ ├── TestCustomS3Async2OutputHandlerIntegration.java │ │ ├── TestCustomS3AsyncOutputHandlerIntegration.java │ │ ├── TestEnvironmentProvider.java │ │ ├── TestMemqClientServerIntegration.java │ │ ├── TestMemqClientServerPerf.java │ │ ├── TestMemqReadBrokers.java │ │ └── producer2 │ │ │ ├── TestMemqClientServerIntegration.java │ │ │ └── TestMemqClientServerPerf.java │ │ ├── processing │ │ ├── TestOutputHandler.java │ │ └── bucketing │ │ │ └── TestBucketingTopicProcessor.java │ │ ├── rpc │ │ ├── TestAuditMessage.java │ │ └── TestAuditor.java │ │ └── utils │ │ └── TestCoreUtils.java │ └── resources │ ├── new.test_topic.json │ └── old.test_topic.json └── pom.xml /.github/CODEOWNERS: -------------------------------------------------------------------------------- 1 | # All owners 2 | * @pinterest/logging 3 | -------------------------------------------------------------------------------- /.github/workflows/publish_to_maven_central.yaml: -------------------------------------------------------------------------------- 1 | name: release and push to central 2 | on: 3 | push: 4 | tags: 5 | - '*' 6 | jobs: 7 | publish: 8 | runs-on: ubuntu-latest 9 | environment: publish 10 | steps: 11 | - uses: actions/checkout@v2 12 | - name: Set up Java for publishing to Maven Central Repository 13 | uses: actions/setup-java@v1 14 | with: 15 | java-version: 11 16 | server-id: ossrh 17 | server-username: MAVEN_USERNAME 18 | server-password: MAVEN_PASSWORD 19 | gpg-private-key: ${{ secrets.OSSRH_GPG_SECRET_KEY }} 20 | gpg-passphrase: MAVEN_GPG_PASSPHRASE 21 | - name: build artifact 22 | run: mvn clean package -DskipTests -pl -memq-client-all,-deploy 23 | - name: Publish to the Maven Central Repository 24 | run: | 25 | mvn \ 26 | --no-transfer-progress \ 27 | --batch-mode \ 28 | deploy -DskipTests -pl -memq-client-all,-deploy 29 | env: 30 | MAVEN_USERNAME: ${{ secrets.OSSRH_USERNAME }} 31 | MAVEN_PASSWORD: ${{ secrets.OSSRH_TOKEN }} 32 | MAVEN_GPG_PASSPHRASE: ${{ secrets.OSSRH_GPG_SECRET_KEY_PASSWORD }} -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | *.class 2 | *.ipr 3 | 4 | .eggs/* 5 | .eggs 6 | 7 | *.pyc 8 | Pipfile.lock 9 | *.egg-info 10 | 11 | .DS_Store* 12 | 13 | # Mobile Tools for Java (J2ME) 14 | .mtj.tmp/ 15 | 16 | # Package Files # 17 | *.jar 18 | *.war 19 | *.ear 20 | 21 | dependency-reduced-pom.xml 22 | 23 | node_modules/* 24 | node_modules 25 | yarn.lock 26 | package-lock.json 27 | 28 | dist/ 29 | build/ 30 | 31 | # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml 32 | hs_err_pid* 33 | target/ 34 | 35 | .metadata 36 | bin/ 37 | tmp/ 38 | *.tmp 39 | *.bak 40 | *.swp 41 | *~.nib 42 | local.properties 43 | .settings/ 44 | .loadpath 45 | .recommenders 46 | 47 | # Eclipse Core 48 | .project 49 | 50 | # External tool builders 51 | .externalToolBuilders/ 52 | 53 | # Locally stored "Eclipse launch configurations" 54 | *.launch 55 | 56 | # PyDev specific (Python IDE for Eclipse) 57 | *.pydevproject 58 | 59 | # CDT-specific (C/C++ Development Tooling) 60 | .cproject 61 | 62 | # JDT-specific (Eclipse Java Development Tools) 63 | .classpath 64 | 65 | # Java annotation processor (APT) 66 | .factorypath 67 | 68 | # PDT-specific (PHP Development Tools) 69 | .buildpath 70 | 71 | # sbteclipse plugin 72 | .target 73 | 74 | # Tern plugin 75 | .tern-project 76 | 77 | # TeXlipse plugin 78 | .texlipse 79 | 80 | # STS (Spring Tool Suite) 81 | .springBeans 82 | 83 | # Code Recommenders 84 | .recommenders/ 85 | 86 | # Covers JetBrains IDEs: IntelliJ, RubyMine, PhpStorm, AppCode, PyCharm, CLion, Android Studio and Webstorm 87 | # Reference: https://intellij-support.jetbrains.com/hc/en-us/articles/206544839 88 | 89 | # User-specific stuff: 90 | .idea/workspace.xml 91 | .idea/tasks.xml 92 | .idea/dictionaries 93 | .idea/vcs.xml 94 | .idea/jsLibraryMappings.xml 95 | 96 | # Sensitive or high-churn files: 97 | .idea/dataSources.ids 98 | .idea/dataSources.xml 99 | .idea/dataSources.local.xml 100 | .idea/sqlDataSources.xml 101 | .idea/dynamic.xml 102 | .idea/uiDesigner.xml 103 | 104 | # Gradle: 105 | .idea/gradle.xml 106 | .idea/libraries 107 | 108 | # Mongo Explorer plugin: 109 | .idea/mongoSettings.xml 110 | 111 | ## File-based project format: 112 | *.iws 113 | 114 | ## Plugin-specific files: 115 | 116 | # IntelliJ 117 | /out/ 118 | .idea/ 119 | *.iml 120 | 121 | # mpeltonen/sbt-idea plugin 122 | .idea_modules/ 123 | 124 | # JIRA plugin 125 | atlassian-ide-plugin.xml 126 | 127 | # Crashlytics plugin (for Android Studio and IntelliJ) 128 | com_crashlytics_export_strings.xml 129 | crashlytics.properties 130 | crashlytics-build.properties 131 | fabric.properties 132 | .vscode/launch.json 133 | .vscode/settings.json 134 | -------------------------------------------------------------------------------- /CONTRIBUTORS.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/memq/83f97daaef49ee14394d9227444c2f140df64802/CONTRIBUTORS.md -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # MemQ: An efficient, scalable cloud native PubSub system 2 | 3 | MemQ is a new PubSub system that augments Kafka at Pinterest. It uses a decoupled storage and serving architecture similar to Apache Pulsar and Facebook Logdevice; however, it relies on a pluggable replicated storage layer i.e. Object Store / DFS / NFS for storing data. The net result is a PubSub system that: 4 | 5 | - Handles GB/s traffic 6 | - Independently scales, writes, and reads 7 | - Doesn’t require expensive rebalancing to handle traffic growth 8 | - Is 90% more cost effective than our Kafka footprint 9 | 10 | ![Design](docs/images/design.png) 11 | 12 | ## Getting Started 13 | 14 | Checkout our [Getting Started](docs/gettingstarted.md) Guide 15 | 16 | ## Usage 17 | 18 | MemQ can be a valuable PubSub choice for the following use case families: 19 | 20 | - Large scale data ingestion 21 | - Bulk uploads 22 | - Near Real Time Analytics 23 | 24 | ## Design 25 | 26 | Detailed architecture docs can be found [here](docs/architecture.md) 27 | 28 | ## Blog 29 | 30 | Please checkout the [original MemQ blog](https://medium.com/pinterest-engineering/memq-an-efficient-scalable-cloud-native-pubsub-system-4402695dd4e7) for details and background. 31 | 32 | ## Authors 33 | 34 | [Ambud Sharma](https://github.com/ambud) 35 | 36 | [Ping Min-Lin](https://github.com/kabochya) 37 | 38 | See [contributors](CONTRIBUTORS.md) 39 | 40 | ## License 41 | 42 | MemQ distributed under [Apache License, Version 2.0](LICENSE) 43 | -------------------------------------------------------------------------------- /deploy/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 | } -------------------------------------------------------------------------------- /deploy/configs/clustered.yaml: -------------------------------------------------------------------------------- 1 | logging: 2 | level: INFO 3 | appenders: 4 | - type: file 5 | threshold: INFO 6 | currentLogFilename: /var/log/memq/memq.log 7 | archivedLogFilenamePattern: /var/log/memq/memq-%i.log.gz 8 | archivedFileCount: 10 9 | maxFileSize: 100MiB 10 | defaultBufferSize: 2097152 11 | defaultSlotTimeout: 300000 12 | clusteringConfig: 13 | enableExpiration: true 14 | zookeeperConnectionString: localhost:2181/memq 15 | cluster: true 16 | resetEnabled: false 17 | awsUploadRatePerStreamInMB: 10 -------------------------------------------------------------------------------- /deploy/configs/nonclustered.yaml: -------------------------------------------------------------------------------- 1 | defaultBufferSize: 2097152 2 | defaultSlotTimeout: 300000 3 | cluster: false 4 | awsUploadRatePerStreamInMB: 10 5 | environmentProvider: com.pinterest.memq.core.config.LocalEnvironmentProvider 6 | topicConfig: 7 | - topic: test 8 | ringBufferSize: 1024 9 | outputParallelism: 4 10 | batchSizeMB: 1 11 | batchMilliSeconds: 50 12 | storageHandlerName: devnull 13 | storageHandlerConfig: 14 | retryTimeoutMillis: 5000 15 | maxAttempts: 2 16 | disableNotifications: true 17 | bucket: 18 | path: test/TestTopic 19 | region: us-east-1 20 | notificationServerset: /tmp/serverset/notification.serverset 21 | notificationTopic: TestTopicNotifications -------------------------------------------------------------------------------- /deploy/pom.xml: -------------------------------------------------------------------------------- 1 | 4 | 4.0.0 5 | 6 | com.pinterest.memq 7 | memq-parent 8 | 0.2.22-SNAPSHOT 9 | ../pom.xml 10 | 11 | memq-deploy 12 | 13 | 14 | com.pinterest.memq 15 | memq 16 | ${project.version} 17 | 18 | 19 | 20 | memq 21 | 22 | 23 | org.apache.maven.plugins 24 | maven-compiler-plugin 25 | 3.8.1 26 | 27 | 1.8 28 | 1.8 29 | 30 | 31 | 32 | org.apache.maven.plugins 33 | maven-shade-plugin 34 | 1.6 35 | 36 | true 37 | 38 | 39 | *:* 40 | 41 | META-INF/*.SF 42 | META-INF/*.DSA 43 | META-INF/*.RSA 44 | 45 | 46 | 47 | 48 | 49 | 50 | 51 | shade 52 | 53 | package 54 | 55 | 56 | 58 | 60 | com.pinterest.memq.core.MemqMain 61 | 62 | 63 | 64 | 65 | 66 | 67 | 68 | 69 | -------------------------------------------------------------------------------- /deploy/run.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | if [ -z $HEAP_MEMORY_GB ];then 4 | export HEAP_MEMORY_GB=1 5 | fi 6 | 7 | # Set JAVA_HOME 8 | JAVA_HOME=${JAVA_HOME:/usr/lib/jvm/openjdk8} 9 | 10 | #export MEM_KB=$(grep MemTotal /proc/meminfo | /usr/bin/awk '{print $2}') 11 | export MEM_KB=$(grep MemTotal /proc/meminfo | tr ' ' '\n' | tail -n2 | head -n1) 12 | export MEM_GB=$((MEM_KB/1024/1024)) 13 | export LOG_DIR=/var/log/memq 14 | 15 | mkdir p $LOG_DIR 16 | 17 | if [ -z $MAX_DIRECT_MEMORY ]; then 18 | echo "Total memory $MEM_GB" 19 | export MAX_DIRECT_MEMORY=$((MEM_GB - HEAP_MEMORY_GB - 4))g 20 | fi 21 | 22 | if [ -n "$ENABLE_NMT" ]; then 23 | echo "Enabling Java NativeMemoryTracking=${ENABLE_NMT}" 24 | export ADDITIONAL_JAVA_OPTS="-XX:NativeMemoryTracking=${ENABLE_NMT}" 25 | fi 26 | 27 | echo "Launching Memq with HEAP=$HEAP_MEMORY_GB MaxDirectMemorySize=$MAX_DIRECT_MEMORY" 28 | $JAVA_HOME/bin/java -Xms${HEAP_MEMORY_GB}g -Xmx${HEAP_MEMORY_GB}g -XX:MaxDirectMemorySize=${MAX_DIRECT_MEMORY} \ 29 | -verbosegc -Xloggc:${LOG_DIR}/gc.log ${ADDITIONAL_JAVA_OPTS} \ 30 | -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=100 -XX:GCLogFileSize=2M \ 31 | -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:+PrintGCDateStamps -XX:+PrintClassHistogram \ 32 | -XX:+UseG1GC -XX:MaxGCPauseMillis=250 -XX:G1ReservePercent=10 -XX:ConcGCThreads=4 \ 33 | -XX:ParallelGCThreads=4 -XX:G1HeapRegionSize=32m -XX:InitiatingHeapOccupancyPercent=70 \ 34 | -XX:ErrorFile=${LOG_DIR}/jvm_error.log \ 35 | -Dsun.net.spi.nameservice.provider.1=dns,sun \ 36 | -jar target/memq.jar server configs/nonclustered.yaml -------------------------------------------------------------------------------- /deploy/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.4.1 12 | if [[ ! -f kafka_2.11-2.4.1.tgz ]]; then 13 | wget https://archive.apache.org/dist/kafka/2.4.1/kafka_2.11-2.4.1.tgz 14 | fi 15 | tar xf kafka_2.11-2.4.1.tgz 16 | 17 | 18 | cd kafka_2.11-2.4.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 TestTopicNotifications" 26 | bin/kafka-topics.sh --zookeeper localhost:2181 --topic TestTopicNotifications --partitions 3 --replication-factor 1 --create 27 | mkdir -p /tmp/serverset 28 | echo "localhost:9092" > /tmp/serverset/notification.serverset -------------------------------------------------------------------------------- /deploy/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.4.1 10 | bin/kafka-server-stop.sh 11 | sleep 2 12 | bin/zookeeper-server-stop.sh -------------------------------------------------------------------------------- /docs/brokerconfig.md: -------------------------------------------------------------------------------- 1 | # MemQ Broker Config 2 | 3 | -------------------------------------------------------------------------------- /docs/deploy.md: -------------------------------------------------------------------------------- 1 | # Deploying MemQ 2 | Below is the guide on how to deployment MemQ. We recommend you read the entire guide before starting to avoid any blockers as there are external systems MemQ depends on. 3 | 4 | ## Pre-requisites 5 | 6 | The following systems need to be deployed and running before we can deploy a MemQ cluster. We have also added links to how to deploy these clusters but are not going to go into how to do that in this guide. 7 | 8 | - [Kafka cluster](https://kafka.apache.org/quickstart) 9 | - [Amazon S3 Bucket](https://aws.amazon.com/getting-started/) 10 | - [Zookeeper cluster](https://zookeeper.apache.org/doc/r3.3.3/zookeeperAdmin.html) (*if using clustered mode*) 11 | 12 | **Note: When creating S3 bucket please make sure to provision IAM Instance Profile credentials for the MemQ Broker hosts and MemQ Consumer** 13 | 14 | ### Non-clustered Deployment (simple deployment) 15 | See `deploy/configs/nonclustered.yaml` 16 | 17 | ### Clustered Deployment (complex deployment) 18 | See `deploy/configus/clustered.yaml` 19 | 20 | #### S3 IAM Configuration Pointers 21 | 22 | Below is a sample terraform template for MemQ IAM policy, the same policy can be used for Broker and Consumer. You can also tweak Consumer policy to make it limit it to `Get*` and `List*` 23 | 24 | Please don't forget to attach your policy to your IAM role. 25 | 26 | ``` 27 | resource "aws_iam_policy" "MyMemQPolicy" { 28 | name = "MyMemQPolicy" 29 | path = "/" 30 | description = "" 31 | 32 | policy = <", 47 | "arn:aws:s3:::/*" 48 | ], 49 | "Sid": "MemQAccess" 50 | } 51 | ], 52 | "Version": "2012-10-17" 53 | } 54 | 55 | POLICY 56 | } 57 | ``` 58 | -------------------------------------------------------------------------------- /docs/images/design.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/memq/83f97daaef49ee14394d9227444c2f140df64802/docs/images/design.png -------------------------------------------------------------------------------- /docs/images/producer.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/memq/83f97daaef49ee14394d9227444c2f140df64802/docs/images/producer.png -------------------------------------------------------------------------------- /docs/images/protocol.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/memq/83f97daaef49ee14394d9227444c2f140df64802/docs/images/protocol.png -------------------------------------------------------------------------------- /docs/opensource.md: -------------------------------------------------------------------------------- 1 | # Open Source 2 | 3 | ## How to release the project to maven central? 4 | - Make sure you are using personal github account instead of enterprise one. 5 | - Update the version in all POM files. 6 | - They look like `#.#.#` 7 | - PR the change and merge into mainline. 8 | - `git push origin tag ` to create the tag 9 | - `git tag ` to tag your change 10 | - `git push` to update the remote repository 11 | - A new workflow should be triggered 12 | - The publish-to-maven workflow can be found here: https://github.com/pinterest/memq/actions. 13 | - It's automatically triggered by a push with tag. Script: https://github.com/pinterest/memq/blob/main/.github/workflows/publish_to_maven_central.yaml 14 | - Find someone to approve the workflow, then the workflow should be able to process. 15 | - After 2-6 hours, the new version should be able to load from maven central. -------------------------------------------------------------------------------- /memq-client-all/.gitignore: -------------------------------------------------------------------------------- 1 | /target/ 2 | -------------------------------------------------------------------------------- /memq-client-all/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | com.pinterest.memq 8 | memq-parent 9 | 0.2.22-SNAPSHOT 10 | ../pom.xml 11 | 12 | memq-client-all 13 | memq-client-all 14 | 15 | UTF-8 16 | 17 | 18 | 19 | com.pinterest.memq 20 | memq-client 21 | 0.2.22-SNAPSHOT 22 | 23 | 24 | 25 | 26 | 27 | org.apache.maven.plugins 28 | maven-compiler-plugin 29 | 3.8.1 30 | 31 | 1.8 32 | 1.8 33 | 34 | 35 | 36 | org.apache.maven.plugins 37 | maven-jar-plugin 38 | 3.2.0 39 | 40 | 41 | 42 | test-jar 43 | 44 | 45 | 46 | 47 | 48 | org.apache.maven.plugins 49 | maven-source-plugin 50 | 3.2.1 51 | 52 | 53 | attach-sources 54 | 55 | jar 56 | 57 | 58 | 59 | 60 | 61 | org.apache.maven.plugins 62 | maven-javadoc-plugin 63 | 3.2.0 64 | 65 | 66 | attach-javadocs 67 | 68 | jar 69 | 70 | 71 | 72 | 73 | 74 | org.apache.maven.plugins 75 | maven-shade-plugin 76 | 3.2.4 77 | 78 | 79 | package 80 | 81 | shade 82 | 83 | 84 | ${basedir}/target/dependency-reduced-pom.xml 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | -------------------------------------------------------------------------------- /memq-client/.gitignore: -------------------------------------------------------------------------------- 1 | /target/ 2 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/AuditorUtils.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons; 17 | 18 | import java.io.IOException; 19 | import java.util.Map.Entry; 20 | import java.util.Properties; 21 | 22 | public class AuditorUtils { 23 | 24 | public static Properties extractAuditorConfig(Properties properties) throws IOException { 25 | Properties auditProps = new Properties(); 26 | for (Entry entry : properties.entrySet()) { 27 | String key = entry.getKey().toString(); 28 | if (key.startsWith(CommonConfigs.AUDITOR_CONFIG_PREFIX)) { 29 | auditProps.put(key.replace(CommonConfigs.AUDITOR_CONFIG_PREFIX, ""), entry.getValue()); 30 | } 31 | } 32 | return auditProps; 33 | } 34 | 35 | } 36 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/CommonConfigs.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons; 17 | 18 | public class CommonConfigs { 19 | 20 | public static final String CLUSTER = "cluster"; 21 | public static final String SERVERSET_FILE = "serverset.file"; 22 | public static final String BOOTSTRAP_SERVERS = "bootstrap.servers"; 23 | public static final String CLIENT_LOCALITY = "client.locality"; 24 | 25 | public static final String AUDITOR_CONFIG_PREFIX = "auditor."; 26 | public static final String AUDITOR_ENABLED = AUDITOR_CONFIG_PREFIX + "enabled"; 27 | public static final String AUDITOR_CLASS = AUDITOR_CONFIG_PREFIX + "class"; 28 | 29 | } 30 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/CommonUtils.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons; 17 | 18 | import java.io.DataInputStream; 19 | import java.io.IOException; 20 | import java.io.InputStream; 21 | import java.util.zip.CRC32; 22 | 23 | public class CommonUtils { 24 | 25 | /** 26 | * Validate whether CRC checksum for a batch matches the given headerCrc 27 | * 28 | * @param batch 29 | * @param headerCrc 30 | * @return 31 | */ 32 | public static boolean crcChecksumMatches(byte[] batch, int headerCrc) { 33 | CRC32 crc = new CRC32(); 34 | crc.update(batch); 35 | if ((int) crc.getValue() != headerCrc) { 36 | return false; 37 | } 38 | return true; 39 | } 40 | 41 | /** 42 | * Given a compression id and compressed InputStream, return an uncompressed 43 | * DataInputStream 44 | * 45 | * @param compression the compression id 46 | * @param original the compressed InputStream 47 | * @return an uncompressed DataInputStream 48 | * @throws IOException 49 | * @throws UnknownCompressionException 50 | */ 51 | public static DataInputStream getUncompressedInputStream(byte compression, 52 | InputStream original) throws IOException, 53 | UnknownCompressionException { 54 | if (compression == 0) { 55 | return new DataInputStream(original); 56 | } 57 | for (Compression comp : Compression.values()) { 58 | if (comp.id == compression) { 59 | return new DataInputStream(comp.getCompressStream(original)); 60 | } 61 | } 62 | throw new UnknownCompressionException("Compression id " + compression + " is not supported"); 63 | } 64 | 65 | } 66 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/Compression.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons; 17 | 18 | import java.io.IOException; 19 | import java.io.InputStream; 20 | import java.io.OutputStream; 21 | import java.util.zip.GZIPInputStream; 22 | import java.util.zip.GZIPOutputStream; 23 | 24 | import com.github.luben.zstd.RecyclingBufferPool; 25 | import com.github.luben.zstd.ZstdInputStreamNoFinalizer; 26 | import com.github.luben.zstd.ZstdOutputStreamNoFinalizer; 27 | 28 | public enum Compression { 29 | NONE(0, 0, is -> is, os -> os), 30 | GZIP(1, 512, 31 | GZIPInputStream::new, 32 | outputStream -> new GZIPOutputStream(outputStream, true) 33 | ), 34 | ZSTD(2, 0, 35 | is -> new ZstdInputStreamNoFinalizer(is, RecyclingBufferPool.INSTANCE), 36 | os -> new ZstdOutputStreamNoFinalizer(os, RecyclingBufferPool.INSTANCE) 37 | ); 38 | 39 | public byte id; 40 | public int minBufferSize; 41 | private final CompressionWrapper inputStreamCompressionWrapper; 42 | private final CompressionWrapper outputStreamCompressionWrapper; 43 | 44 | 45 | Compression(int id, int minBufferSize, 46 | CompressionWrapper inputStreamCompressionWrapper, 47 | CompressionWrapper outputStreamCompressionWrapper) { 48 | this.id = (byte) id; 49 | this.minBufferSize = minBufferSize; 50 | this.inputStreamCompressionWrapper = inputStreamCompressionWrapper; 51 | this.outputStreamCompressionWrapper = outputStreamCompressionWrapper; 52 | } 53 | 54 | @FunctionalInterface 55 | private interface CompressionWrapper { 56 | T getWrappedInstance(T t) throws IOException; 57 | } 58 | 59 | public InputStream getCompressStream(InputStream is) throws IOException { 60 | return inputStreamCompressionWrapper.getWrappedInstance(is); 61 | } 62 | 63 | public OutputStream getDecompressStream(OutputStream os) throws IOException { 64 | return outputStreamCompressionWrapper.getWrappedInstance(os); 65 | } 66 | } -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/ConsumerConfigs.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons; 17 | 18 | public class ConsumerConfigs extends CommonConfigs { 19 | 20 | public static final String CLIENT_ID = "clientId"; 21 | public static final String USE_STREAMING_ITERATOR = "useStreamingIterator"; 22 | public static final String NOTIFICATION_SOURCE_TYPE_KEY = "notificationSourceType"; 23 | public static final String NOTIFICATION_SOURCE_PROPS_KEY = "notificationSourceProps"; 24 | public static final String NOTIFICATION_SOURCE_PROPS_PREFIX_KEY = "notification."; 25 | public static final String STORAGE_PROPS_PREFIX_KEY = "storage."; 26 | public static final String BUFFER_TO_FILE_CONFIG_KEY = "bufferToFile"; 27 | public static final String USE_DIRECT_BUFFER_KEY = "directBuffer"; 28 | public static final String BUFFER_FILES_DIRECTORY_KEY = "bufferFilename"; 29 | public static final String KEY_DESERIALIZER_CLASS_KEY = "key.deserializerclass"; 30 | public static final String VALUE_DESERIALIZER_CLASS_KEY = "value.deserializerclass"; 31 | public static final String KEY_DESERIALIZER_CLASS_CONFIGS_KEY = "key.deserializerclass.configs"; 32 | public static final String VALUE_DESERIALIZER_CLASS_CONFIGS_KEY = "value.deserializerclass.configs"; 33 | public static final String AUTO_COMMIT_PER_POLL_KEY = "autoCommitPerPoll"; 34 | public static final String DRY_RUN_KEY = "dryRun"; 35 | public static final String DIRECT_CONSUMER = "directConsumer"; 36 | public static final String GROUP_ID = "group.id"; 37 | public static final String TOPIC_INTERNAL_PROP = "topic"; 38 | 39 | } 40 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/DataCorruptionException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons; 17 | 18 | public class DataCorruptionException extends Exception { 19 | 20 | private static final long serialVersionUID = 1L; 21 | 22 | public DataCorruptionException(String message) { 23 | super(message); 24 | } 25 | } -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/Deserializer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons; 17 | 18 | import java.util.Properties; 19 | 20 | public interface Deserializer { 21 | 22 | public default void init(Properties props) { 23 | } 24 | 25 | public T deserialize(byte[] bytes); 26 | 27 | } 28 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/MemqNettyClientSideResponseHandler.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons; 17 | 18 | import org.slf4j.Logger; 19 | import org.slf4j.LoggerFactory; 20 | 21 | import com.pinterest.memq.commons.protocol.RequestType; 22 | import com.pinterest.memq.commons.protocol.ResponsePacket; 23 | 24 | import io.netty.buffer.ByteBuf; 25 | import io.netty.channel.ChannelHandlerContext; 26 | import io.netty.channel.ChannelInboundHandlerAdapter; 27 | 28 | public class MemqNettyClientSideResponseHandler extends ChannelInboundHandlerAdapter { 29 | 30 | private static final Logger logger = LoggerFactory 31 | .getLogger(MemqNettyClientSideResponseHandler.class); 32 | private ResponseHandler responseHandler; 33 | 34 | public MemqNettyClientSideResponseHandler(ResponseHandler responseHandler) { 35 | this.responseHandler = responseHandler; 36 | } 37 | 38 | @Override 39 | public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { 40 | ByteBuf buf = (ByteBuf) msg; 41 | try { 42 | ResponsePacket responsePacket = new ResponsePacket(); 43 | responsePacket.readFields(buf, RequestType.PROTOCOL_VERSION); 44 | logger.debug("Response received {}", responsePacket); 45 | if (responsePacket.getProtocolVersion() != RequestType.PROTOCOL_VERSION) { 46 | // might not be able to handle this request. 47 | // in future multiple protocol versions can / should be handled here 48 | logger.debug("Server responded in protocol different than client request:{} vs {}", 49 | responsePacket.getProtocolVersion(), RequestType.PROTOCOL_VERSION); 50 | } else { 51 | responseHandler.handle(responsePacket); 52 | } 53 | } catch (Exception e) { 54 | logger.error("Failed to handle server responses", e); 55 | throw e; 56 | } finally { 57 | buf.release(); 58 | } 59 | } 60 | 61 | } 62 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/MemqWriteResult.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons; 17 | 18 | public class MemqWriteResult { 19 | 20 | private long clientRequestId; 21 | private int writeLatency; 22 | private int ackLatency; 23 | private int bytesWritten; 24 | 25 | public MemqWriteResult() { 26 | } 27 | 28 | public MemqWriteResult(long clientRequestId, int writeLatency, int ackLatency, int bytesWritten) { 29 | super(); 30 | this.writeLatency = writeLatency; 31 | this.ackLatency = ackLatency; 32 | this.bytesWritten = bytesWritten; 33 | } 34 | 35 | public int getWriteLatency() { 36 | return writeLatency; 37 | } 38 | 39 | public void setWriteLatency(int writeLatency) { 40 | this.writeLatency = writeLatency; 41 | } 42 | 43 | public int getAckLatency() { 44 | return ackLatency; 45 | } 46 | 47 | public void setAckLatency(int ackLatency) { 48 | this.ackLatency = ackLatency; 49 | } 50 | 51 | public int getBytesWritten() { 52 | return bytesWritten; 53 | } 54 | 55 | public void setBytesWritten(int bytesWritten) { 56 | this.bytesWritten = bytesWritten; 57 | } 58 | 59 | public long getClientRequestId() { 60 | return clientRequestId; 61 | } 62 | 63 | } 64 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/ProducerConfigs.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons; 17 | 18 | public class ProducerConfigs extends CommonConfigs { 19 | 20 | public static final String TOPIC_NAME = "topic.name"; 21 | public static final String VALUE_SERIALIZER = "value.serializer"; 22 | public static final String KEY_SERIALIZER = "key.serializer"; 23 | public static final String CLIENT_TYPE = "client.type"; 24 | public static final String REQUEST_ACKS_TIMEOUT_MS = "request.acks.timeout.ms"; 25 | public static final String REQUEST_ACKS_CHECKPOLLINTERVAL_MS = "request.acks.checkpollinterval.ms"; 26 | public static final String REQUEST_ACKS_DISABLE = "request.acks.disable"; 27 | public static final String REQUEST_COMPRESSION_TYPE = "request.compression.type"; 28 | public static final String REQUEST_MAX_PAYLOADBYTES = "request.max.payloadbytes"; 29 | public static final String REQUEST_MAX_INFLIGHTREQUESTS = "request.max.inflightrequests"; 30 | public static final String REQUEST_TIMEOUT = "request.timeout"; 31 | 32 | // Defaults 33 | public static final String DEFAULT_REQUEST_ACKS_TIMEOUT_MS = "60000"; 34 | public static final String DEFAULT_ACK_CHECKPOLLINTERVAL_MS = "100"; 35 | public static final String DEFAULT_DISABLE_ACKS = "false"; 36 | public static final String DEFAULT_COMPRESSION_TYPE = Compression.ZSTD.name(); 37 | public static final String DEFAULT_MAX_PAYLOADBYTES = String.valueOf(1024 * 1024); 38 | public static final String DEFAULT_MAX_INFLIGHT_REQUESTS = "30"; 39 | public static final String DEFAULT_LOCALITY = "none"; 40 | 41 | } 42 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/ResponseHandler.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons; 17 | 18 | import java.util.Map; 19 | import java.util.function.Consumer; 20 | 21 | import org.slf4j.Logger; 22 | import org.slf4j.LoggerFactory; 23 | 24 | import com.pinterest.memq.commons.protocol.ResponsePacket; 25 | 26 | public class ResponseHandler { 27 | 28 | private static final Logger logger = LoggerFactory.getLogger(ResponseHandler.class); 29 | private Map> requestMap; 30 | 31 | public ResponseHandler() { 32 | } 33 | 34 | public void handle(ResponsePacket responsePacket) throws Exception { 35 | Consumer consumer = requestMap 36 | .remove(MemqCommonClient.makeResponseKey(responsePacket)); 37 | if (consumer != null) { 38 | consumer.accept(responsePacket); 39 | } else { 40 | // no handler for response skipping 41 | logger.error("No handler for request:" + responsePacket.getRequestType()); 42 | } 43 | } 44 | 45 | public void setRequestMap(Map> requestMap) { 46 | this.requestMap = requestMap; 47 | } 48 | 49 | } 50 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/SimpleMessageId.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons; 17 | 18 | import java.nio.ByteBuffer; 19 | 20 | import com.pinterest.memq.commons.MessageId; 21 | 22 | public class SimpleMessageId extends MessageId { 23 | 24 | public SimpleMessageId(long id) { 25 | super(ByteBuffer.allocate(8).putLong(id).array()); 26 | } 27 | 28 | } 29 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/UnknownCompressionException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons; 17 | 18 | public class UnknownCompressionException extends Exception { 19 | 20 | private static final long serialVersionUID = 1L; 21 | 22 | public UnknownCompressionException(String message) { 23 | super(message); 24 | } 25 | } -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/audit/Auditor.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons.audit; 17 | 18 | import java.io.IOException; 19 | import java.util.Properties; 20 | 21 | public abstract class Auditor { 22 | 23 | public Auditor() { 24 | } 25 | 26 | public abstract void init(Properties props) throws Exception; 27 | 28 | public abstract void auditMessage(byte[] cluster, 29 | byte[] topic, 30 | byte[] hostAddress, 31 | long epoch, 32 | long id, 33 | byte[] hash, 34 | int count, 35 | boolean isProducer, 36 | String clientId) throws IOException; 37 | 38 | public abstract void close(); 39 | } 40 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/serde/ByteArrayDeserializer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons.serde; 17 | 18 | import com.pinterest.memq.client.commons.Deserializer; 19 | 20 | public class ByteArrayDeserializer implements Deserializer { 21 | 22 | @Override 23 | public byte[] deserialize(byte[] bytes) { 24 | return bytes; 25 | } 26 | 27 | } 28 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/serde/ByteArraySerializer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons.serde; 17 | 18 | public class ByteArraySerializer implements Serializer { 19 | 20 | @Override 21 | public byte[] serialize(byte[] data) { 22 | return data; 23 | } 24 | 25 | } 26 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/serde/Serializer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons.serde; 17 | 18 | public interface Serializer { 19 | 20 | public byte[] serialize(T data); 21 | 22 | } 23 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/serde/StringDeserializer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons.serde; 17 | 18 | import com.pinterest.memq.client.commons.Deserializer; 19 | 20 | public class StringDeserializer implements Deserializer { 21 | 22 | @Override 23 | public String deserialize(byte[] bytes) { 24 | return new String(bytes); 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons/serde/ThriftDeserializer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons.serde; 17 | 18 | import java.util.Properties; 19 | 20 | import org.apache.thrift.TBase; 21 | import org.apache.thrift.TDeserializer; 22 | import org.apache.thrift.TException; 23 | 24 | import com.pinterest.memq.client.commons.Deserializer; 25 | 26 | @SuppressWarnings("rawtypes") 27 | public class ThriftDeserializer implements Deserializer { 28 | 29 | public static final String TBASE_OBJECT_CONFIG = "tBaseObject"; 30 | private TBase tbase; 31 | 32 | @Override 33 | public void init(Properties props) { 34 | if (!props.containsKey(TBASE_OBJECT_CONFIG)) { 35 | throw new RuntimeException("ThriftDeserializer must have TBASE_OBJECT_CONFIG"); 36 | } 37 | tbase = (TBase) props.get(TBASE_OBJECT_CONFIG); 38 | } 39 | 40 | @Override 41 | public TBase deserialize(byte[] bytes) { 42 | try { 43 | TDeserializer tDeserializer = new TDeserializer(); 44 | tDeserializer.deserialize(tbase, bytes); 45 | return tbase; 46 | } catch (TException e) { 47 | throw new RuntimeException(e); 48 | } 49 | } 50 | 51 | } 52 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons2/ClosedException.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.memq.client.commons2; 2 | 3 | public class ClosedException extends Exception { 4 | private static final long serialVersionUID = 1L; 5 | } 6 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons2/DataNotFoundException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons2; 17 | 18 | public class DataNotFoundException extends Exception { 19 | 20 | private static final long serialVersionUID = 1L; 21 | 22 | public DataNotFoundException() { 23 | super(); 24 | } 25 | 26 | public DataNotFoundException(String message, 27 | Throwable cause, 28 | boolean enableSuppression, 29 | boolean writableStackTrace) { 30 | super(message, cause, enableSuppression, writableStackTrace); 31 | } 32 | 33 | public DataNotFoundException(String message, Throwable cause) { 34 | super(message, cause); 35 | } 36 | 37 | public DataNotFoundException(String message) { 38 | super(message); 39 | } 40 | 41 | public DataNotFoundException(Throwable cause) { 42 | super(cause); 43 | } 44 | 45 | } 46 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons2/Endpoint.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons2; 17 | 18 | import com.pinterest.memq.commons.protocol.Broker; 19 | 20 | import java.net.InetSocketAddress; 21 | 22 | public class Endpoint { 23 | 24 | public static final String DEFAULT_LOCALITY = "n/a"; 25 | private InetSocketAddress address; 26 | private String locality = DEFAULT_LOCALITY; 27 | 28 | public Endpoint(InetSocketAddress address, String locality) { 29 | this.address = address; 30 | this.locality = locality; 31 | } 32 | 33 | public Endpoint(InetSocketAddress address) { 34 | this.address = address; 35 | } 36 | 37 | public InetSocketAddress getAddress() { 38 | return address; 39 | } 40 | 41 | public void setAddress(InetSocketAddress address) { 42 | this.address = address; 43 | } 44 | 45 | public String getLocality() { 46 | return locality; 47 | } 48 | 49 | public void setLocality(String locality) { 50 | this.locality = locality; 51 | } 52 | 53 | public static Endpoint fromBroker(Broker broker) { 54 | return new Endpoint(InetSocketAddress.createUnresolved(broker.getBrokerIP(), 55 | broker.getBrokerPort()), broker.getLocality()); 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons2/TopicNotFoundException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons2; 17 | 18 | public class TopicNotFoundException extends Exception { 19 | 20 | private static final long serialVersionUID = 1L; 21 | 22 | public TopicNotFoundException() { 23 | } 24 | 25 | public TopicNotFoundException(String message) { 26 | super(message); 27 | } 28 | 29 | public TopicNotFoundException(String message, Throwable cause) { 30 | super(message, cause); 31 | } 32 | 33 | public TopicNotFoundException(Throwable cause) { 34 | super(cause); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons2/TransportPacketIdentifier.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons2; 17 | 18 | import com.pinterest.memq.commons.protocol.TransportPacket; 19 | 20 | import io.netty.buffer.ByteBuf; 21 | 22 | import java.io.IOException; 23 | 24 | public class TransportPacketIdentifier extends TransportPacket { 25 | 26 | public TransportPacketIdentifier(TransportPacket packet) { 27 | protocolVersion = packet.getProtocolVersion(); 28 | clientRequestId = packet.getClientRequestId(); 29 | requestType = packet.getRequestType(); 30 | } 31 | 32 | @Override 33 | public boolean equals(Object o) { 34 | if (this == o) { 35 | return true; 36 | } 37 | if (o == null || getClass() != o.getClass()) { 38 | return false; 39 | } 40 | 41 | TransportPacketIdentifier that = (TransportPacketIdentifier) o; 42 | 43 | if (protocolVersion != that.protocolVersion) { 44 | return false; 45 | } 46 | if (clientRequestId != that.clientRequestId) { 47 | return false; 48 | } 49 | return requestType == that.requestType; 50 | } 51 | 52 | @Override 53 | public int hashCode() { 54 | int result = protocolVersion; 55 | result = 31 * result + (int) (clientRequestId ^ (clientRequestId >>> 32)); 56 | result = 31 * result + requestType.hashCode(); 57 | return result; 58 | } 59 | 60 | @Override 61 | public String toString() { 62 | return "PacketIdentifier{" + 63 | "protocolVersion=" + protocolVersion + 64 | ", clientId=" + clientRequestId + 65 | ", requestType=" + requestType + 66 | '}'; 67 | } 68 | 69 | @Override 70 | public void readFields(ByteBuf buf, short protocolVersion) throws IOException { 71 | 72 | } 73 | 74 | @Override 75 | public void write(ByteBuf buf, short protocolVersion) { 76 | 77 | } 78 | 79 | @Override 80 | public int getSize(short protocolVersion) { 81 | return 0; 82 | } 83 | } 84 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons2/network/ClientClosedException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons2.network; 17 | 18 | public class ClientClosedException extends Exception { 19 | 20 | private static final long serialVersionUID = 1L; 21 | 22 | public ClientClosedException() { 23 | } 24 | 25 | public ClientClosedException(String message) { 26 | super(message); 27 | } 28 | 29 | public ClientClosedException(String message, Throwable cause) { 30 | super(message, cause); 31 | } 32 | 33 | public ClientClosedException(Throwable cause) { 34 | super(cause); 35 | } 36 | 37 | public ClientClosedException(String message, Throwable cause, boolean enableSuppression, 38 | boolean writableStackTrace) { 39 | super(message, cause, enableSuppression, writableStackTrace); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons2/network/ClosedConnectionException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons2.network; 17 | 18 | import java.io.IOException; 19 | 20 | public class ClosedConnectionException extends IOException { 21 | 22 | private static final long serialVersionUID = 1L; 23 | 24 | public ClosedConnectionException() { 25 | } 26 | 27 | public ClosedConnectionException(String message) { 28 | super(message); 29 | } 30 | 31 | public ClosedConnectionException(String message, Throwable cause) { 32 | super(message, cause); 33 | } 34 | 35 | public ClosedConnectionException(Throwable cause) { 36 | super(cause); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons2/network/netty/MemqNettyClientSideResponseHandler.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons2.network.netty; 17 | 18 | import com.pinterest.memq.client.commons2.network.ResponseHandler; 19 | import com.pinterest.memq.commons.protocol.RequestType; 20 | import com.pinterest.memq.commons.protocol.ResponsePacket; 21 | 22 | import io.netty.buffer.ByteBuf; 23 | import io.netty.channel.ChannelHandlerContext; 24 | import io.netty.channel.ChannelInboundHandlerAdapter; 25 | import org.slf4j.Logger; 26 | import org.slf4j.LoggerFactory; 27 | 28 | public class MemqNettyClientSideResponseHandler extends ChannelInboundHandlerAdapter { 29 | 30 | private static final Logger logger = LoggerFactory.getLogger(MemqNettyClientSideResponseHandler.class); 31 | private ResponseHandler responseHandler; 32 | 33 | public MemqNettyClientSideResponseHandler(ResponseHandler responseHandler) { 34 | this.responseHandler = responseHandler; 35 | } 36 | 37 | @Override 38 | public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { 39 | ByteBuf buf = (ByteBuf) msg; 40 | try { 41 | ResponsePacket responsePacket = new ResponsePacket(); 42 | responsePacket.readFields(buf, RequestType.PROTOCOL_VERSION); 43 | logger.debug("Response received " + responsePacket); 44 | if (responsePacket.getProtocolVersion() != RequestType.PROTOCOL_VERSION) { 45 | // might not be able to handle this request. 46 | // in future multiple protocol versions can / should be handled here 47 | logger.debug("Server responded in protocol different than client request: " +responsePacket.getProtocolVersion() + " vs " + RequestType.PROTOCOL_VERSION); 48 | } else { 49 | responseHandler.handle(responsePacket); 50 | } 51 | } catch (Exception e) { 52 | logger.error("Failed to handle server responses: ", e); 53 | throw e; 54 | } finally { 55 | buf.release(); 56 | } 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons2/retry/ExponentialBackoffRetryStrategy.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons2.retry; 17 | 18 | import java.time.Duration; 19 | 20 | public class ExponentialBackoffRetryStrategy 21 | extends RetryStrategy { 22 | private int maxRetries = 2; 23 | private long baseRetryIntervalMs = 500; 24 | 25 | public ExponentialBackoffRetryStrategy() { 26 | } 27 | 28 | public ExponentialBackoffRetryStrategy(int maxRetries, long baseRetryIntervalMs) { 29 | this.maxRetries = maxRetries; 30 | this.baseRetryIntervalMs = baseRetryIntervalMs; 31 | } 32 | 33 | public void setMaxRetries(int maxRetries) { 34 | this.maxRetries = maxRetries; 35 | } 36 | 37 | public void setBaseRetryIntervalMs(long baseRetryIntervalMs) { 38 | this.baseRetryIntervalMs = baseRetryIntervalMs; 39 | } 40 | 41 | @Override 42 | public Duration calculateNextRetryInterval(int attempts) { 43 | if (attempts >= maxRetries) { 44 | return null; 45 | } 46 | return Duration.ofMillis(Math.round(baseRetryIntervalMs * Math.pow(2, attempts))); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons2/retry/FullJitterRetryStrategy.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons2.retry; 17 | 18 | import java.time.Duration; 19 | import java.util.concurrent.ThreadLocalRandom; 20 | 21 | public class FullJitterRetryStrategy extends RetryStrategy { 22 | 23 | private long baseRetryIntervalMs = 200; 24 | private long maxRetryIntervalMs = 10000; 25 | private int maxAttempts = Integer.MAX_VALUE - 1; 26 | 27 | public FullJitterRetryStrategy() { 28 | } 29 | 30 | public void setBaseRetryIntervalMs(long baseRetryIntervalMs) { 31 | this.baseRetryIntervalMs = baseRetryIntervalMs; 32 | } 33 | 34 | public void setMaxRetryIntervalMs(long maxRetryIntervalMs) { 35 | this.maxRetryIntervalMs = maxRetryIntervalMs; 36 | } 37 | 38 | public void setMaxAttempts(int maxAttempts) { 39 | this.maxAttempts = maxAttempts; 40 | } 41 | 42 | @Override 43 | public Duration calculateNextRetryInterval(int attempts) { 44 | if (attempts >= maxAttempts ) { 45 | return null; 46 | } 47 | long upper = Math.min((Math.round(Math.pow(2, attempts) * baseRetryIntervalMs)), maxRetryIntervalMs); 48 | return Duration.ofMillis(ThreadLocalRandom.current().nextLong(1, upper + 1)); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons2/retry/RetryStrategy.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons2.retry; 17 | 18 | import java.time.Duration; 19 | 20 | public abstract class RetryStrategy { 21 | 22 | /** 23 | * returns an interval till the next attempt 24 | * @param attempts number of attempts that are already done 25 | * @return null if no retries are left, the next interval otherwise 26 | */ 27 | public abstract Duration calculateNextRetryInterval(int attempts); 28 | } 29 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/commons2/retry/UniformRetryStrategy.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons2.retry; 17 | 18 | import java.time.Duration; 19 | 20 | public class UniformRetryStrategy extends RetryStrategy { 21 | private int retryIntervalMs = 1000; 22 | 23 | public UniformRetryStrategy() { 24 | } 25 | 26 | public void setRetryIntervalMs(int retryIntervalMs) { 27 | this.retryIntervalMs = retryIntervalMs; 28 | } 29 | 30 | @Override 31 | public Duration calculateNextRetryInterval(int attempts) { 32 | return Duration.ofMillis(retryIntervalMs); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/consumer/NoTopicsSubscribedException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.consumer; 17 | 18 | public class NoTopicsSubscribedException extends Exception { 19 | 20 | private static final long serialVersionUID = 1L; 21 | 22 | public NoTopicsSubscribedException(String message) { 23 | super(message); 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/consumer/NotificationSource.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.consumer; 17 | 18 | import java.time.Duration; 19 | import java.util.Collection; 20 | import java.util.List; 21 | import java.util.Map; 22 | import java.util.Queue; 23 | import java.util.Set; 24 | import java.util.concurrent.TimeoutException; 25 | 26 | import org.apache.kafka.common.PartitionInfo; 27 | import org.apache.kafka.common.TopicPartition; 28 | 29 | import com.google.gson.JsonObject; 30 | 31 | interface NotificationSource { 32 | 33 | int lookForNewObjects(Duration timeout, Queue notificationQueue); 34 | 35 | void assign(Collection asList); 36 | 37 | void seek(Map notificationOffset); 38 | 39 | long position(int partition); 40 | 41 | long committed(int partition); 42 | 43 | public void commit(Map offsetMap); 44 | 45 | void commit(); 46 | 47 | void commitAsync(); 48 | 49 | void commitAsync(OffsetCommitCallback callback); 50 | 51 | void commitAsync(Map offsets, OffsetCommitCallback callback); 52 | 53 | Object getRawObject(); 54 | 55 | void unsubscribe(); 56 | 57 | void close(); 58 | 59 | String getNotificationTopicName(); 60 | 61 | List getPartitions(); 62 | 63 | void setParentConsumer(MemqConsumer memqConsumer); 64 | 65 | Map offsetsForTimestamps(Map partitionTimestamps); 66 | 67 | Map getEarliestOffsets(Collection partitions); 68 | 69 | Map getLatestOffsets(Collection partitions); 70 | 71 | Set waitForAssignment(); 72 | 73 | void wakeup(); 74 | 75 | Map getNotificationsAtOffsets(Duration timeout, 76 | Map partitionOffsets) throws TimeoutException; 77 | 78 | Set getAssignments(); 79 | 80 | void seekToEnd(Collection partitions); 81 | 82 | void seekToBeginning(Collection partitions); 83 | 84 | } 85 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/consumer/OffsetCommitCallback.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.memq.client.consumer; 2 | 3 | import java.util.Map; 4 | 5 | public interface OffsetCommitCallback { 6 | void onCompletion(Map offsets, Exception exception); 7 | } 8 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/consumer/utils/CheckNotificationTopic.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.consumer.utils; 17 | 18 | import java.io.FileInputStream; 19 | import java.util.Properties; 20 | 21 | import com.pinterest.memq.client.consumer.KafkaNotificationSource; 22 | 23 | public class CheckNotificationTopic { 24 | 25 | public static void main(String[] args) throws Exception { 26 | Properties props = new Properties(); 27 | props.load(new FileInputStream(args[0])); 28 | KafkaNotificationSource source = new KafkaNotificationSource(props); 29 | source.unsubscribe(); 30 | long ts = System.currentTimeMillis(); 31 | long[] offsetsForAllPartitions = source.getOffsetsForAllPartitions(true); 32 | for (int i = 0; i < offsetsForAllPartitions.length; i++) { 33 | long l = offsetsForAllPartitions[i]; 34 | System.out.println("Partition:" + i + " Offset:" + l); 35 | } 36 | ts = System.currentTimeMillis() - ts; 37 | System.out.println("Ts:" + ts); 38 | } 39 | 40 | } 41 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/consumer/utils/properties/PropertiesInitializationException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.consumer.utils.properties; 17 | 18 | public class PropertiesInitializationException extends Exception { 19 | 20 | private static final long serialVersionUID = 1L; 21 | 22 | public PropertiesInitializationException(String message) { 23 | super(message); 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/producer/MemqWriteResult.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.producer; 17 | 18 | public class MemqWriteResult { 19 | 20 | private long clientRequestId; 21 | private int writeLatency; 22 | private int ackLatency; 23 | private int bytesWritten; 24 | 25 | public MemqWriteResult() { 26 | } 27 | 28 | public MemqWriteResult(long clientRequestId, int writeLatency, int ackLatency, int bytesWritten) { 29 | this.clientRequestId = clientRequestId; 30 | this.writeLatency = writeLatency; 31 | this.ackLatency = ackLatency; 32 | this.bytesWritten = bytesWritten; 33 | } 34 | 35 | public int getWriteLatency() { 36 | return writeLatency; 37 | } 38 | 39 | public void setWriteLatency(int writeLatency) { 40 | this.writeLatency = writeLatency; 41 | } 42 | 43 | public int getAckLatency() { 44 | return ackLatency; 45 | } 46 | 47 | public void setAckLatency(int ackLatency) { 48 | this.ackLatency = ackLatency; 49 | } 50 | 51 | public int getBytesWritten() { 52 | return bytesWritten; 53 | } 54 | 55 | public void setBytesWritten(int bytesWritten) { 56 | this.bytesWritten = bytesWritten; 57 | } 58 | 59 | public long getClientRequestId() { 60 | return clientRequestId; 61 | } 62 | 63 | } 64 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/client/producer/http/DaemonThreadFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.producer.http; 17 | 18 | import java.util.concurrent.ThreadFactory; 19 | import java.util.concurrent.atomic.AtomicInteger; 20 | 21 | public final class DaemonThreadFactory implements ThreadFactory { 22 | 23 | private String basename; 24 | private AtomicInteger counter; 25 | 26 | public DaemonThreadFactory(String basename) { 27 | this.basename = basename; 28 | this.counter = new AtomicInteger(); 29 | } 30 | 31 | @Override 32 | public Thread newThread(Runnable r) { 33 | Thread th = new Thread(r); 34 | th.setDaemon(true); 35 | th.setName(basename + "-" + counter.getAndIncrement()); 36 | return th; 37 | } 38 | } -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/commons/storage/DelayedDevNullStorageHandler.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.storage; 17 | 18 | import java.io.IOException; 19 | import java.util.List; 20 | import java.util.Properties; 21 | import java.util.concurrent.ThreadLocalRandom; 22 | import java.util.concurrent.atomic.AtomicLong; 23 | 24 | import com.codahale.metrics.MetricRegistry; 25 | import com.pinterest.memq.core.commons.Message; 26 | import com.pinterest.memq.core.commons.MessageBufferInputStream; 27 | 28 | @StorageHandlerName(name = "delayeddevnull") 29 | public class DelayedDevNullStorageHandler implements StorageHandler { 30 | 31 | private int maxDelay; 32 | private ThreadLocalRandom rand; 33 | private int minDelay; 34 | private static AtomicLong counter = new AtomicLong(); 35 | private static AtomicLong byteCounter = new AtomicLong(); 36 | private static AtomicLong inputStreamCounter = new AtomicLong(); 37 | 38 | @Override 39 | public void initWriter(Properties outputHandlerConfig, 40 | String topic, 41 | MetricRegistry registry) throws Exception { 42 | minDelay = Integer.parseInt(outputHandlerConfig.getProperty("delay.min.millis", "100")); 43 | maxDelay = Integer.parseInt(outputHandlerConfig.getProperty("delay.max.millis", "2000")); 44 | rand = ThreadLocalRandom.current(); 45 | } 46 | 47 | @Override 48 | public void writeOutput(int sizeInBytes, 49 | int checksum, 50 | List messages) throws WriteFailedException { 51 | try { 52 | Thread.sleep(rand.nextInt(minDelay, maxDelay)); 53 | counter.accumulateAndGet(messages.size(), (v1, v2) -> v1 + v2); 54 | byteCounter.accumulateAndGet(sizeInBytes, (v1, v2) -> v1 + v2); 55 | MessageBufferInputStream is = new MessageBufferInputStream(messages, null); 56 | while (is.read() != -1) { 57 | inputStreamCounter.incrementAndGet(); 58 | } 59 | is.close(); 60 | } catch (InterruptedException | IOException e) { 61 | // ignore errors 62 | } 63 | } 64 | 65 | public static long getCounter() { 66 | return counter.get(); 67 | } 68 | 69 | public static long getByteCounter() { 70 | return byteCounter.get(); 71 | } 72 | 73 | public static long getInputStreamCounter() { 74 | return inputStreamCounter.get(); 75 | } 76 | 77 | public static void reset() { 78 | counter.set(0); 79 | byteCounter.set(0); 80 | inputStreamCounter.set(0); 81 | } 82 | 83 | @Override 84 | public String getReadUrl() { 85 | return "delayeddevnull"; 86 | } 87 | } -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/commons/storage/DevNullStorageHandler.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.storage; 17 | 18 | import com.pinterest.memq.commons.storage.s3.KafkaNotificationSink; 19 | import com.pinterest.memq.core.commons.Message; 20 | import com.pinterest.memq.core.utils.MiscUtils; 21 | 22 | import com.codahale.metrics.MetricRegistry; 23 | import com.codahale.metrics.Timer; 24 | import com.google.gson.JsonObject; 25 | import io.netty.buffer.ByteBuf; 26 | 27 | import java.io.File; 28 | import java.nio.channels.FileChannel; 29 | import java.nio.file.StandardOpenOption; 30 | import java.util.List; 31 | import java.util.Properties; 32 | 33 | @StorageHandlerName(name = "devnull") 34 | public class DevNullStorageHandler implements StorageHandler { 35 | 36 | private KafkaNotificationSink notificationSink; 37 | private boolean disableNotifications; 38 | private Timer notificationPublishingTimer; 39 | 40 | @Override 41 | public void initWriter(Properties outputHandlerConfig, String topic, MetricRegistry registry) 42 | throws Exception { 43 | this.disableNotifications = Boolean 44 | .parseBoolean(outputHandlerConfig.getProperty("disableNotifications", "true")); 45 | if (!disableNotifications) { 46 | this.notificationSink = new KafkaNotificationSink(); 47 | this.notificationSink.init(outputHandlerConfig); 48 | this.notificationPublishingTimer = MiscUtils.oneMinuteWindowTimer(registry,"output.notification.publish.latency"); 49 | } 50 | } 51 | 52 | @Override 53 | public void writeOutput(int sizeInBytes, int checksum, List messages) 54 | throws WriteFailedException { 55 | File fileToWrite = new File("/dev/null"); 56 | 57 | ByteBuf batchHeader = StorageHandler.getBatchHeadersAsByteArray(messages); 58 | 59 | try (FileChannel fc = FileChannel.open(fileToWrite.toPath(), StandardOpenOption.WRITE)) { 60 | batchHeader.readBytes(fc, batchHeader.readableBytes()); 61 | for (Message m : messages) { 62 | ByteBuf buf = m.getBuf(); 63 | buf.readBytes(fc, buf.readableBytes()); 64 | } 65 | 66 | if (!disableNotifications) { 67 | JsonObject payload = new JsonObject(); 68 | payload.addProperty("type", "devnull"); 69 | Timer.Context publishTime = notificationPublishingTimer.time(); 70 | notificationSink.notify(payload, 0); 71 | publishTime.stop(); 72 | } 73 | } catch (Exception e) { 74 | throw new WriteFailedException(e); 75 | } finally { 76 | batchHeader.release(); 77 | } 78 | } 79 | 80 | @Override 81 | public String getReadUrl() { 82 | return "devnull"; 83 | } 84 | } 85 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/commons/storage/NoOpStorageHandler.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.storage; 17 | 18 | import com.pinterest.memq.core.commons.Message; 19 | 20 | import com.codahale.metrics.MetricRegistry; 21 | 22 | import java.util.List; 23 | import java.util.Properties; 24 | 25 | @StorageHandlerName(name = "noop") 26 | public class NoOpStorageHandler implements StorageHandler { 27 | @Override 28 | public void initWriter(Properties outputHandlerConfig, 29 | String topic, 30 | MetricRegistry registry) throws Exception { 31 | } 32 | 33 | @Override 34 | public void writeOutput(int sizeInBytes, 35 | int checksum, 36 | List messages) throws WriteFailedException { 37 | 38 | } 39 | @Override 40 | public String getReadUrl() { 41 | return "noop"; 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/commons/storage/StorageHandlerName.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.storage; 17 | 18 | import java.lang.annotation.ElementType; 19 | import java.lang.annotation.Retention; 20 | import java.lang.annotation.RetentionPolicy; 21 | import java.lang.annotation.Target; 22 | 23 | @Retention(RetentionPolicy.RUNTIME) 24 | @Target({ ElementType.TYPE }) 25 | public @interface StorageHandlerName { 26 | 27 | String name(); 28 | 29 | String previousName() default ""; 30 | 31 | } 32 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/commons/storage/StorageHandlerTable.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.storage; 17 | 18 | import java.util.HashMap; 19 | import java.util.Map; 20 | import java.util.Set; 21 | import java.util.logging.Logger; 22 | 23 | import org.reflections.Reflections; 24 | 25 | /** 26 | * Ideal inherited from: 27 | * https://github.com/srotya/sidewinder/blob/development/core/src/main/java/com/srotya/sidewinder/core/functions/FunctionTable.java 28 | * 29 | */ 30 | public abstract class StorageHandlerTable { 31 | 32 | private static final Logger logger = Logger.getLogger(StorageHandlerTable.class.getName()); 33 | 34 | private static Map> handlerMap = new HashMap<>(); 35 | 36 | static { 37 | findAndRegisterOutputHandlers(StorageHandlerTable.class.getPackage().getName()); 38 | } 39 | 40 | public static void findAndRegisterOutputHandlers(String packageName) { 41 | Reflections reflections = new Reflections(packageName.trim()); 42 | Set> annotatedClasses = reflections.getTypesAnnotatedWith(StorageHandlerName.class); 43 | for (Class annotatedClass : annotatedClasses) { 44 | StorageHandlerName plugin = annotatedClass.getAnnotation(StorageHandlerName.class); 45 | if (plugin == null) { 46 | logger.severe("Plugin info null:" + plugin); 47 | continue; 48 | } 49 | registerStorageHandlerClassWithAlias(annotatedClass, plugin.name()); 50 | registerStorageHandlerClassWithAlias(annotatedClass, plugin.previousName()); 51 | } 52 | } 53 | 54 | @SuppressWarnings("unchecked") 55 | private static void registerStorageHandlerClassWithAlias(Class annotatedClass, String alias) { 56 | if (alias == null || alias.isEmpty()) { 57 | logger.warning("Ignoring aggregation function:" + annotatedClass.getName()); 58 | return; 59 | } 60 | if (handlerMap.containsKey(alias)) { 61 | logger.severe( 62 | "Output plugin alias '" + alias + "' already exists, " + annotatedClass.getName()); 63 | System.exit(-1); 64 | } 65 | handlerMap.put(alias, (Class) annotatedClass); 66 | logger 67 | .info("Registered output handler(" + annotatedClass.getName() + ") with alias:" + alias); 68 | } 69 | 70 | @SuppressWarnings("unchecked") 71 | public static Class getClass(String name) { 72 | return (Class) handlerMap.get(name); 73 | } 74 | 75 | } 76 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/commons/storage/SysoutStorageHandler.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.storage; 17 | 18 | import java.util.List; 19 | import java.util.Properties; 20 | 21 | import com.codahale.metrics.MetricRegistry; 22 | import com.pinterest.memq.core.commons.Message; 23 | 24 | import io.netty.buffer.ByteBuf; 25 | 26 | @StorageHandlerName(name = "sysout") 27 | public class SysoutStorageHandler implements StorageHandler { 28 | 29 | @Override 30 | public void initWriter(Properties outputHandlerConfig, 31 | String topic, 32 | MetricRegistry registry) throws Exception { 33 | } 34 | 35 | @Override 36 | public void writeOutput(int sizeInBytes, 37 | int checksum, 38 | List messages) throws WriteFailedException { 39 | for (Message message : messages) { 40 | System.out.println(new String(readToByteArray(message.getBuf()))); 41 | } 42 | } 43 | 44 | public static byte[] readToByteArray(ByteBuf buf) { 45 | byte[] ary = new byte[buf.readableBytes()]; 46 | for (int i = 0; i < buf.readableBytes(); i++) { 47 | ary[i] = buf.readByte(); 48 | } 49 | return ary; 50 | } 51 | 52 | @Override 53 | public String getReadUrl() { 54 | return "System.in"; 55 | } 56 | 57 | } -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/commons/storage/WriteFailedException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.storage; 17 | 18 | public class WriteFailedException extends Exception { 19 | 20 | private static final long serialVersionUID = 1L; 21 | 22 | public WriteFailedException(String msg) { 23 | super(msg); 24 | } 25 | 26 | public WriteFailedException(Exception e) { 27 | super(e); 28 | } 29 | 30 | } 31 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/commons/storage/s3/MemqS3DNSResolver.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.storage.s3; 17 | 18 | import java.net.InetAddress; 19 | import java.net.UnknownHostException; 20 | import java.util.Arrays; 21 | import java.util.logging.Logger; 22 | 23 | import org.apache.http.conn.DnsResolver; 24 | 25 | public class MemqS3DNSResolver implements DnsResolver { 26 | 27 | private static final Logger logger = Logger.getLogger(MemqS3DNSResolver.class.getCanonicalName()); 28 | 29 | @Override 30 | public InetAddress[] resolve(String host) throws UnknownHostException { 31 | InetAddress[] address = InetAddress.getAllByName(host); 32 | logger.fine(() -> "Host:" + host + " address:" + Arrays.toString(address)); 33 | return address; 34 | } 35 | } -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/commons/storage/s3/S3Exception.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.storage.s3; 17 | 18 | import java.io.IOException; 19 | 20 | public abstract class S3Exception extends IOException { 21 | 22 | private static final long serialVersionUID = 1L; 23 | 24 | private int errorCode; 25 | 26 | public S3Exception() { 27 | } 28 | 29 | public S3Exception(int errorCode) { 30 | super("S3 Exception (code: " + errorCode + ")"); 31 | this.errorCode = errorCode; 32 | } 33 | 34 | public S3Exception(String message) { 35 | super(message); 36 | } 37 | 38 | public S3Exception(int errorCode, String message) { 39 | super("S3 Exception (code: " + errorCode + ", message: " + message + ")"); 40 | this.errorCode = errorCode; 41 | } 42 | 43 | public int getErrorCode() { 44 | return errorCode; 45 | } 46 | 47 | public void setErrorCode(int errorCode) { 48 | this.errorCode = errorCode; 49 | } 50 | 51 | public static class RetriableException extends S3Exception { 52 | 53 | public RetriableException(int errorCode) { 54 | super(errorCode); 55 | } 56 | 57 | private static final long serialVersionUID = 1L; 58 | 59 | } 60 | 61 | public static class NotFoundException extends S3Exception { 62 | 63 | public NotFoundException() { 64 | super(404); 65 | } 66 | 67 | private static final long serialVersionUID = 1L; 68 | 69 | } 70 | 71 | public static class ForbiddenException extends S3Exception { 72 | 73 | public ForbiddenException() { 74 | super(403); 75 | } 76 | 77 | private static final long serialVersionUID = 1L; 78 | 79 | } 80 | 81 | public static class InternalServerErrorException extends RetriableException { 82 | 83 | public InternalServerErrorException() { 84 | super(500); 85 | } 86 | 87 | private static final long serialVersionUID = 1L; 88 | 89 | } 90 | 91 | public static class ServiceUnavailableException extends RetriableException { 92 | 93 | public ServiceUnavailableException() { 94 | super(503); 95 | } 96 | 97 | private static final long serialVersionUID = 1L; 98 | 99 | } 100 | 101 | } 102 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/commons/storage/s3/reader/client/RequestClient.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.storage.s3.reader.client; 17 | 18 | import software.amazon.awssdk.http.SdkHttpFullRequest; 19 | import software.amazon.awssdk.services.s3.model.GetObjectRequest; 20 | 21 | import java.io.Closeable; 22 | import java.io.IOException; 23 | import java.io.InputStream; 24 | import java.util.Properties; 25 | 26 | import io.netty.buffer.ByteBuf; 27 | 28 | public interface RequestClient extends Closeable { 29 | void initialize(Properties properties); 30 | InputStream tryObjectGet(SdkHttpFullRequest request) throws IOException; 31 | ByteBuf tryObjectGetAsBuffer(SdkHttpFullRequest request) throws IOException; 32 | 33 | InputStream tryObjectGet(GetObjectRequest request) throws IOException; 34 | ByteBuf tryObjectGetAsBuffer(GetObjectRequest request) throws IOException; 35 | } 36 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/commons/storage/s3express/S3ExpressHelper.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.memq.commons.storage.s3express; 2 | 3 | import java.util.HashMap; 4 | import java.util.Map; 5 | 6 | public class S3ExpressHelper { 7 | public static class S3ExpressParsingException extends Exception { 8 | public S3ExpressParsingException(String message) { 9 | super(message); 10 | } 11 | } 12 | 13 | /** 14 | * Map from region code to AWS region name 15 | * The region code is the second part of the bucket name, e.g. "use1" in "s3express--use1--us-east-1--x-s3" 16 | * The region name is the AWS region name, e.g. "us-east-1" 17 | */ 18 | public static final Map awsRegionMap = new HashMap() {{ 19 | put("use1", "us-east-1"); 20 | }}; 21 | 22 | /** 23 | * Validate the bucket name is a valid s3express bucket name 24 | * https://docs.aws.amazon.com/AmazonS3/latest/userguide/bucketnamingrules.html#bucketnamingrules-directorybucket 25 | * @param bucketName the bucket name to validate 26 | * @throws S3ExpressParsingException 27 | */ 28 | public static void validateS3ExpressBucketName(String bucketName) throws S3ExpressParsingException { 29 | if (!bucketName.matches(".*--.*-.*--x-s3")) { 30 | throw new S3ExpressParsingException("Invalid s3express bucket name: " + bucketName); 31 | } 32 | } 33 | 34 | /** 35 | * Generate the bucket URL from the bucket name 36 | * @param bucketName 37 | * @return the bucket URL 38 | * @throws S3ExpressParsingException 39 | */ 40 | public static String generateBucketUrl(String bucketName) throws S3ExpressParsingException{ 41 | validateS3ExpressBucketName(bucketName); 42 | String region = getRegionFromBucket(bucketName); 43 | String azName = bucketName.split("--")[1]; 44 | return String.format("https://%s.s3express-%s.%s.amazonaws.com/", bucketName, azName, region); 45 | } 46 | 47 | /** 48 | * Get the region name from the bucket name 49 | * @param bucketName 50 | * @return the region name 51 | * @throws S3ExpressParsingException 52 | */ 53 | public static String getRegionFromBucket(String bucketName) throws S3ExpressParsingException { 54 | validateS3ExpressBucketName(bucketName); 55 | String regionCode = bucketName.split("--")[1].split("-")[0]; 56 | if (!awsRegionMap.containsKey(regionCode)) { 57 | throw new S3ExpressParsingException( 58 | String.format("Unknown region code %s from bucket name %s", regionCode, bucketName)); 59 | } 60 | return awsRegionMap.get(regionCode); 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/commons/storage/s3express/SessionCreds.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2024 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.storage.s3express; 17 | 18 | /** 19 | * Hold AWS Session Credentials 20 | */ 21 | public class SessionCreds { 22 | 23 | public String key; 24 | public String secret; 25 | public String token; 26 | 27 | public SessionCreds() { 28 | } 29 | 30 | public SessionCreds(String key, String secret, String token) { 31 | this.key = key; 32 | this.secret = secret; 33 | this.token = token; 34 | } 35 | 36 | public void setKey(String key) { 37 | this.key = key; 38 | } 39 | 40 | public void setSecret(String secret) { 41 | this.secret = secret; 42 | } 43 | 44 | public void setToken(String token) { 45 | this.token = token; 46 | } 47 | 48 | public String getKey() { 49 | return key; 50 | } 51 | 52 | public String getSecret() { 53 | return secret; 54 | } 55 | 56 | public String getToken() { 57 | return token; 58 | } 59 | 60 | @Override 61 | public String toString() { 62 | return "SessionCreds [key=" + key + ", secret=" + secret + ", token=" + token + "]"; 63 | } 64 | 65 | } 66 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/commons/storage/s3express/keygenerator/DateHourKeyGenerator.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.memq.commons.storage.s3express.keygenerator; 2 | 3 | import java.text.SimpleDateFormat; 4 | 5 | /** 6 | * Generate S3 object key with date and hour as prefix 7 | * The key format is: yyMMdd-HH/{path}/{firstMessageClientRequestId}_{firstMessageServerRequestId}_{attempt} 8 | * For example: 240101-01/test_topic/123_456_1 9 | * 10 | * The date-hour prefix can help the cleaning job to clean up the old data. 11 | * Until 2024/11/05, AWS S3Express does not support object lifecycle policy. 12 | * We need to clean up the old data manually or via scripts. 13 | * With this setup, we can easily clean up the old data by deleting the hourly prefix. 14 | */ 15 | public class DateHourKeyGenerator extends S3ExpressObjectKeyGenerator { 16 | 17 | private static final String DATE_HOUR_PATTERN = "yyMMdd-HH"; 18 | 19 | public DateHourKeyGenerator(String path) { 20 | super(path); 21 | } 22 | 23 | @Override 24 | public String generateObjectKey(long firstMessageClientRequestId, 25 | long firstMessageServerRequestId, 26 | int attempt) { 27 | StringBuilder keyBuilder = new StringBuilder(); 28 | keyBuilder.append(getCurrentDateHr()); 29 | keyBuilder.append(SLASH); 30 | keyBuilder.append(path); 31 | keyBuilder.append(SLASH); 32 | keyBuilder.append(firstMessageClientRequestId); 33 | keyBuilder.append(SEPARATOR); 34 | keyBuilder.append(firstMessageServerRequestId); 35 | keyBuilder.append(SEPARATOR); 36 | keyBuilder.append(attempt); 37 | return keyBuilder.toString(); 38 | } 39 | 40 | protected static String getCurrentDateHr() { 41 | return new SimpleDateFormat(DATE_HOUR_PATTERN).format(new java.util.Date()); 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /memq-client/src/main/java/com/pinterest/memq/commons/storage/s3express/keygenerator/S3ExpressObjectKeyGenerator.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.memq.commons.storage.s3express.keygenerator; 2 | 3 | public abstract class S3ExpressObjectKeyGenerator { 4 | 5 | protected static final String SLASH = "/"; 6 | protected static final String SEPARATOR = "_"; 7 | 8 | protected String path; 9 | 10 | public S3ExpressObjectKeyGenerator(String path) { 11 | this.path = path; 12 | } 13 | 14 | /** 15 | * Generate the S3Express object key 16 | * @param firstMessageClientRequestId 17 | * @param firstMessageServerRequestId 18 | * @param attempt 19 | * @return 20 | */ 21 | public abstract String generateObjectKey(long firstMessageClientRequestId, 22 | long firstMessageServerRequestId, 23 | int attempt); 24 | } 25 | -------------------------------------------------------------------------------- /memq-client/src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # log4j logging dev configuration. 2 | 3 | # root logger. 4 | log4j.rootLogger=DEBUG, CONSOLE, ROLLINGFILE 5 | 6 | log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender 7 | log4j.appender.CONSOLE.Threshold=INFO 8 | log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout 9 | log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} [%t] (%C:%L) %-5p %m%n 10 | 11 | log4j.appender.ROLLINGFILE=org.apache.log4j.RollingFileAppender 12 | log4j.appender.ROLLINGFILE.Threshold=DEBUG 13 | log4j.appender.ROLLINGFILE.File=/tmp/singer.log 14 | # keep log files up to 20MB 15 | log4j.appender.ROLLINGFILE.MaxFileSize=20MB 16 | log4j.appender.ROLLINGFILE.MaxBackupIndex=50 17 | log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout 18 | log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} [%t] (%C:%L) %-5p %m%n 19 | -------------------------------------------------------------------------------- /memq-client/src/test/java/com/pinterest/memq/client/commons/ZstdTest.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.client.commons; 17 | 18 | import static org.junit.Assert.assertTrue; 19 | 20 | import java.io.FileInputStream; 21 | import java.io.FileOutputStream; 22 | import java.io.InputStream; 23 | import java.io.OutputStream; 24 | import java.util.Random; 25 | 26 | import org.apache.commons.io.IOUtils; 27 | import org.apache.commons.io.output.TeeOutputStream; 28 | 29 | import com.github.luben.zstd.ZstdInputStream; 30 | import com.github.luben.zstd.ZstdOutputStream; 31 | 32 | // Used to verify zstd version compatibility 33 | public class ZstdTest { 34 | 35 | //@Test 36 | public void write() throws Exception { 37 | FileOutputStream originalFile = new FileOutputStream("test-original"); 38 | FileOutputStream compressedFile = new FileOutputStream("test-compressed"); 39 | OutputStream compressionStream = new ZstdOutputStream(compressedFile); 40 | byte[] bytes = new byte[2 * 1024 * 1024]; 41 | new Random().nextBytes(bytes); 42 | TeeOutputStream tos = new TeeOutputStream(originalFile, compressionStream); 43 | tos.write(bytes); 44 | tos.close(); 45 | } 46 | 47 | //@Test 48 | public void read() throws Exception { 49 | FileInputStream originalFile = new FileInputStream("test-original"); 50 | FileInputStream compressedFile = new FileInputStream("test-compressed"); 51 | InputStream decompressionStream = new ZstdInputStream(compressedFile); 52 | assertTrue(IOUtils.contentEquals(originalFile, decompressionStream)); 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /memq-client/src/test/java/com/pinterest/memq/commons/storage/s3/TestAbstractS3StorageHandler.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.storage.s3; 17 | 18 | import static org.junit.Assert.assertEquals; 19 | 20 | import org.junit.Test; 21 | 22 | import reactor.netty.http.client.HttpClient; 23 | import reactor.netty.http.client.HttpClientResponse; 24 | 25 | public class TestAbstractS3StorageHandler { 26 | 27 | @Test 28 | public void testAmazonCertMigration() { 29 | HttpClient client = HttpClient.create().secure(); 30 | HttpClientResponse block = client.get() 31 | .uri("https://s3-ats-migration-test.s3.eu-west-3.amazonaws.com/test.jpg").response() 32 | .block(); 33 | int code = block.status().code(); 34 | assertEquals(200, code); 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /memq-client/src/test/java/com/pinterest/memq/commons/storage/s3/TestCustomS3Async2OutputHandler.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.storage.s3; 17 | 18 | import static org.junit.Assert.assertEquals; 19 | import static org.junit.Assert.assertTrue; 20 | import static org.junit.Assert.fail; 21 | 22 | import java.time.Duration; 23 | import java.util.ArrayList; 24 | import java.util.List; 25 | import java.util.Properties; 26 | import java.util.concurrent.CompletableFuture; 27 | import java.util.concurrent.ExecutionException; 28 | import java.util.concurrent.TimeoutException; 29 | 30 | import org.junit.Test; 31 | 32 | import com.codahale.metrics.MetricRegistry; 33 | 34 | public class TestCustomS3Async2OutputHandler { 35 | 36 | @Test 37 | public void testAnyUploadResultOrTimeout() throws Exception { 38 | CustomS3Async2StorageHandler handler = new CustomS3Async2StorageHandler(); 39 | Properties props = new Properties(); 40 | props.setProperty("bucket", "test"); 41 | handler.initWriter(props, "test", new MetricRegistry()); 42 | List> tasks = new ArrayList<>(); 43 | 44 | for(int i = 1; i <= 5; i++) { 45 | final int j = i; 46 | tasks.add(CompletableFuture.supplyAsync(() -> { 47 | try { 48 | Thread.sleep(200 * j); 49 | } catch (Exception e) { 50 | } 51 | return new CustomS3Async2StorageHandler.UploadResult("task-" + j, 200, null, 0, j); 52 | })); 53 | } 54 | 55 | try { 56 | CustomS3Async2StorageHandler.UploadResult r = handler.anyUploadResultOrTimeout(tasks, Duration.ofMillis(1000)).get(); 57 | assertEquals(r.getKey(), "task-1"); 58 | } catch (Exception e) { 59 | fail("Should not fail: " + e); 60 | } 61 | 62 | tasks.clear(); 63 | 64 | for(int i = 1; i <= 5; i++) { 65 | final int j = i; 66 | tasks.add(CompletableFuture.supplyAsync(() -> { 67 | try { 68 | Thread.sleep(200 * j + 1000); 69 | } catch (Exception e) { 70 | } 71 | return new CustomS3Async2StorageHandler.UploadResult("task-" + j, 200, null, 0, j); 72 | })); 73 | } 74 | 75 | try { 76 | CustomS3Async2StorageHandler.UploadResult r = handler.anyUploadResultOrTimeout(tasks, Duration.ofMillis(1000)).get(); 77 | fail("Should timeout"); 78 | } catch (ExecutionException ee) { 79 | System.out.println(ee); 80 | assertTrue(ee.getCause() instanceof TimeoutException); 81 | } catch (Exception e) { 82 | fail("Should throw timeout exception"); 83 | } 84 | } 85 | } -------------------------------------------------------------------------------- /memq-client/src/test/java/com/pinterest/memq/commons/storage/s3express/TestS3ExpressHelper.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.memq.commons.storage.s3express; 2 | 3 | import org.junit.Test; 4 | 5 | import static org.junit.Assert.assertEquals; 6 | 7 | public class TestS3ExpressHelper { 8 | @Test 9 | public void testGenerateBucketUrl() throws Exception { 10 | String bucketName = "testbucket--use1-az5--x-s3"; 11 | assertEquals( 12 | "https://testbucket--use1-az5--x-s3.s3express-use1-az5.us-east-1.amazonaws.com/", 13 | S3ExpressHelper.generateBucketUrl(bucketName) 14 | ); 15 | } 16 | 17 | @Test 18 | public void testValidateS3ExpressBucketName() throws Exception { 19 | String bucketName = "testbucket--use1-az5--x-s3"; 20 | S3ExpressHelper.validateS3ExpressBucketName(bucketName); 21 | } 22 | 23 | @Test (expected = S3ExpressHelper.S3ExpressParsingException.class) 24 | public void testValidateS3ExpressBucketNameInvalid() throws Exception { 25 | String bucketName = "test-bucket"; 26 | S3ExpressHelper.validateS3ExpressBucketName(bucketName); 27 | } 28 | 29 | @Test (expected = S3ExpressHelper.S3ExpressParsingException.class) 30 | public void getRegionFromBucketInvalid() throws Exception { 31 | String bucketName = "testbucket--unknownRegion-az5--x-s3"; 32 | S3ExpressHelper.getRegionFromBucket(bucketName); 33 | } 34 | 35 | @Test 36 | public void testGetRegionFromBucket() throws Exception { 37 | String bucketName = "testbucket--use1-az5--x-s3"; 38 | assertEquals("us-east-1", S3ExpressHelper.getRegionFromBucket(bucketName)); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /memq-client/src/test/java/com/pinterest/memq/commons/storage/s3express/keygenerator/TestDateHourKeyGenerator.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.memq.commons.storage.s3express.keygenerator; 2 | 3 | import org.junit.Test; 4 | 5 | import static org.junit.Assert.assertTrue; 6 | 7 | public class TestDateHourKeyGenerator { 8 | @Test 9 | public void testGetCurrentDateHr() { 10 | String currentDateHr = DateHourKeyGenerator.getCurrentDateHr(); 11 | assertTrue(currentDateHr.matches("\\d{2}\\d{2}\\d{2}-\\d{2}")); 12 | } 13 | } 14 | -------------------------------------------------------------------------------- /memq-commons/.gitignore: -------------------------------------------------------------------------------- 1 | /target/ 2 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/CloseableIterator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons; 17 | 18 | import java.io.Closeable; 19 | import java.util.Iterator; 20 | 21 | public interface CloseableIterator extends Iterator, Closeable { 22 | } 23 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/MessageId.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons; 17 | 18 | public class MessageId { 19 | 20 | protected byte[] array; 21 | 22 | public MessageId(byte[] array) { 23 | this.array = array; 24 | } 25 | 26 | public byte[] toByteArray() { 27 | return array; 28 | } 29 | 30 | public void fromByteArray(byte[] array) { 31 | this.array = array; 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/config/SSLConfig.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.config; 17 | 18 | import java.util.Collections; 19 | import java.util.List; 20 | 21 | /** 22 | * listeners=PLAINTEXT://:9092,SSL://:9093 23 | * security.inter.broker.protocol=PLAINTEXT ssl.client.auth=required 24 | * ssl.enabled.protocols=TLSv1.2,TLSv1.1,TLSv1 25 | * ssl.endpoint.identification.algorithm=HTTPS ssl.key.password=pintastic 26 | * ssl.keystore.location=/var/lib/normandie/fuse/jks/generic 27 | * ssl.keystore.password=pintastic ssl.keystore.type=JKS 28 | * ssl.secure.random.implementation=SHA1PRNG 29 | * ssl.truststore.location=/var/lib/normandie/fuse/jkstrust/generic 30 | * ssl.truststore.password=pintastic ssl.truststore.type=JKS 31 | * authorizer.class.name=com.pinterest.commons.kafka.authorizers.PastisAuthorizer 32 | * kafka.authorizer.pastis_policy=kafka 33 | */ 34 | public class SSLConfig { 35 | 36 | private String keystorePath; 37 | private String keystoreType; 38 | private String keystorePassword; 39 | private String truststorePath; 40 | private String truststoreType; 41 | private String truststorePassword; 42 | private List protocols = Collections.singletonList("TLSv1.2"); 43 | 44 | public String getKeystorePath() { 45 | return keystorePath; 46 | } 47 | 48 | public void setKeystorePath(String keystorePath) { 49 | this.keystorePath = keystorePath; 50 | } 51 | 52 | public String getKeystoreType() { 53 | return keystoreType; 54 | } 55 | 56 | public void setKeystoreType(String keystoreType) { 57 | this.keystoreType = keystoreType; 58 | } 59 | 60 | public String getKeystorePassword() { 61 | return keystorePassword; 62 | } 63 | 64 | public void setKeystorePassword(String keystorePassword) { 65 | this.keystorePassword = keystorePassword; 66 | } 67 | 68 | public String getTruststorePath() { 69 | return truststorePath; 70 | } 71 | 72 | public void setTruststorePath(String truststorePath) { 73 | this.truststorePath = truststorePath; 74 | } 75 | 76 | public String getTruststoreType() { 77 | return truststoreType; 78 | } 79 | 80 | public void setTruststoreType(String truststoreType) { 81 | this.truststoreType = truststoreType; 82 | } 83 | 84 | public String getTruststorePassword() { 85 | return truststorePassword; 86 | } 87 | 88 | public void setTruststorePassword(String truststorePassword) { 89 | this.truststorePassword = truststorePassword; 90 | } 91 | 92 | public List getProtocols() { 93 | return protocols; 94 | } 95 | 96 | public void setProtocols(List protocols) { 97 | this.protocols = protocols; 98 | } 99 | } 100 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/protocol/BatchData.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.protocol; 17 | 18 | import java.io.IOException; 19 | 20 | import io.netty.buffer.ByteBuf; 21 | 22 | public class BatchData implements Packet { 23 | 24 | private ByteBuf dataAsBuf; 25 | private int length; 26 | private Object sendFileRef; 27 | 28 | public BatchData() { 29 | } 30 | 31 | public BatchData(int length, ByteBuf dataBuf) { 32 | this.length = length; 33 | this.dataAsBuf = dataBuf; 34 | } 35 | 36 | public BatchData(int length, Object sendFileRef) { 37 | this.length = length; 38 | this.sendFileRef = sendFileRef; 39 | } 40 | 41 | @Override 42 | public void readFields(ByteBuf buf, short protocolVersion) throws IOException { 43 | length = buf.readInt(); 44 | if (length > 0) { 45 | buf.retain(); 46 | dataAsBuf = buf.readSlice(length); 47 | } 48 | } 49 | 50 | @Override 51 | public void write(ByteBuf buf, short protocolVersion) { 52 | buf.writeInt(length); 53 | if (dataAsBuf != null) { 54 | // this is an optimization hack to enable sendFile calls 55 | // if dataAsBuf is not null then send it using that else just write the length 56 | // and use other workarounds for writing the payload 57 | dataAsBuf.resetReaderIndex(); 58 | buf.writeBytes(dataAsBuf); 59 | } 60 | } 61 | 62 | @Override 63 | public int getSize(short protocolVersion) { 64 | return Integer.BYTES + (length > 0 ? length : 0); 65 | } 66 | 67 | public int getLength() { 68 | return length; 69 | } 70 | 71 | public void setLength(int length) { 72 | this.length = length; 73 | } 74 | 75 | public ByteBuf getDataAsBuf() { 76 | return dataAsBuf; 77 | } 78 | 79 | public void setDataAsBuf(ByteBuf dataAsBuf) { 80 | this.dataAsBuf = dataAsBuf; 81 | } 82 | 83 | public Object getSendFileRef() { 84 | return sendFileRef; 85 | } 86 | 87 | public void setSendFileRef(Object sendFileRef) { 88 | this.sendFileRef = sendFileRef; 89 | } 90 | 91 | } 92 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/protocol/ClusterMetadataResponse.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.protocol; 17 | 18 | import java.io.IOException; 19 | import java.util.HashMap; 20 | import java.util.Map; 21 | 22 | import io.netty.buffer.ByteBuf; 23 | 24 | public class ClusterMetadataResponse implements Packet { 25 | 26 | private Map topicMetadataMap = new HashMap<>(); 27 | 28 | public ClusterMetadataResponse() { 29 | } 30 | 31 | public ClusterMetadataResponse(Map topicMetadataMap) { 32 | this.topicMetadataMap = topicMetadataMap; 33 | } 34 | 35 | @Override 36 | public void readFields(ByteBuf buf, short protocolVersion) throws IOException { 37 | short topicCount = buf.readShort(); 38 | for (int i = 0; i < topicCount; i++) { 39 | String topicName = ProtocolUtils.readStringWithTwoByteEncoding(buf); 40 | TopicMetadata md = new TopicMetadata(); 41 | md.readFields(buf, protocolVersion); 42 | topicMetadataMap.put(topicName, md); 43 | } 44 | } 45 | 46 | @Override 47 | public void write(ByteBuf buf, short protocolVersion) { 48 | 49 | } 50 | 51 | @Override 52 | public int getSize(short protocolVersion) { 53 | return topicMetadataMap.entrySet().stream() 54 | .mapToInt(e -> ProtocolUtils.getStringSerializedSizeWithTwoByteEncoding(e.getKey()) 55 | + e.getValue().getSize(protocolVersion)) 56 | .sum(); 57 | } 58 | 59 | } 60 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/protocol/MemqConstants.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.protocol; 17 | 18 | public class MemqConstants { 19 | public static final int DEFAULT_CRC32_CHECKSUM = 0; 20 | } 21 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/protocol/Packet.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.protocol; 17 | 18 | import java.io.IOException; 19 | 20 | import io.netty.buffer.ByteBuf; 21 | 22 | public interface Packet { 23 | 24 | void readFields(ByteBuf buf, short protocolVersion) throws IOException; 25 | 26 | void write(ByteBuf buf, short protocolVersion); 27 | 28 | int getSize(short protocolVersion); 29 | 30 | default void release() throws IOException { 31 | 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/protocol/ProtocolUtils.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.protocol; 17 | 18 | import io.netty.buffer.ByteBuf; 19 | 20 | public class ProtocolUtils { 21 | 22 | public static String readStringWithTwoByteEncoding(ByteBuf buf) { 23 | short length = buf.readShort(); 24 | byte[] str = new byte[length]; 25 | buf.readBytes(str); 26 | return new String(str); 27 | } 28 | 29 | public static void writeStringWithTwoByteEncoding(ByteBuf buf, String str) { 30 | if (str == null || str.isEmpty()) { 31 | buf.writeShort(0); 32 | } else { 33 | byte[] bytes = str.getBytes(); 34 | buf.writeShort(bytes.length); 35 | buf.writeBytes(bytes); 36 | } 37 | } 38 | 39 | public static int getStringSerializedSizeWithTwoByteEncoding(String str) { 40 | return (str == null || str.isEmpty() ? 0 : (str.getBytes().length)) + Short.BYTES; 41 | } 42 | 43 | } 44 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/protocol/ReadResponsePacket.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.protocol; 17 | 18 | import java.io.IOException; 19 | 20 | import io.netty.buffer.ByteBuf; 21 | 22 | public class ReadResponsePacket implements Packet { 23 | 24 | private BatchData batchData; 25 | 26 | public ReadResponsePacket() { 27 | } 28 | 29 | public ReadResponsePacket(BatchData batchData) { 30 | this.batchData = batchData; 31 | } 32 | 33 | @Override 34 | public void readFields(ByteBuf buf, short protocolVersion) throws IOException { 35 | batchData = new BatchData(); 36 | batchData.readFields(buf, protocolVersion); 37 | } 38 | 39 | @Override 40 | public void write(ByteBuf buf, short protocolVersion) { 41 | batchData.write(buf, protocolVersion); 42 | } 43 | 44 | @Override 45 | public int getSize(short protocolVersion) { 46 | return batchData.getSize(protocolVersion); 47 | } 48 | 49 | public BatchData getBatchData() { 50 | return batchData; 51 | } 52 | 53 | @Override 54 | public String toString() { 55 | return "ReadResponsePacket [batchData=" + batchData + "]"; 56 | } 57 | 58 | } 59 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/protocol/RequestPacket.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.protocol; 17 | 18 | import java.io.IOException; 19 | 20 | import io.netty.buffer.ByteBuf; 21 | 22 | public class RequestPacket extends TransportPacket { 23 | private Packet payload; 24 | 25 | public RequestPacket() { 26 | } 27 | 28 | public RequestPacket(short protocolVersion, 29 | long clientRequestId, 30 | RequestType requestType, 31 | Packet payload) { 32 | super(protocolVersion, clientRequestId, requestType); 33 | this.payload = payload; 34 | } 35 | 36 | @Override 37 | public void readFields(ByteBuf inBuffer, short pv) throws IOException { 38 | inBuffer.readInt(); 39 | this.protocolVersion = inBuffer.readShort(); 40 | clientRequestId = inBuffer.readLong(); 41 | requestType = RequestType.extractPacketType(inBuffer); 42 | payload = requestType.requestImplementationSupplier.get(); 43 | payload.readFields(inBuffer, this.protocolVersion); 44 | } 45 | 46 | @Override 47 | public void write(ByteBuf outBuf, short protocolVersion) { 48 | outBuf.writeInt(getSize(protocolVersion)); 49 | outBuf.writeShort(protocolVersion); 50 | outBuf.writeLong(clientRequestId); 51 | outBuf.writeByte(requestType.ordinal()); 52 | payload.write(outBuf, protocolVersion); 53 | } 54 | 55 | @Override 56 | public int getSize(short protocolVersion) { 57 | return Short.BYTES + Long.BYTES + Byte.BYTES + payload.getSize(protocolVersion); 58 | } 59 | 60 | public Packet getPayload() { 61 | return payload; 62 | } 63 | 64 | public void setPayload(Packet payload) { 65 | this.payload = payload; 66 | } 67 | 68 | @Override 69 | public void release() throws IOException { 70 | payload.release(); 71 | super.release(); 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/protocol/RequestType.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.protocol; 17 | 18 | import java.io.IOException; 19 | import java.util.function.Supplier; 20 | 21 | import io.netty.buffer.ByteBuf; 22 | 23 | /** 24 | * MemQ request type information.
25 | *
26 | * 27 | * NOTE: ENUM ORIDINAL VALUEs used for protocol encoding in protocol. DO NOT 28 | * CHANGE THE ORDERING OF THE ENUM ENTRIES AS THAT WILL CAUSE PROTOCOL 29 | * CORRUPTION. 30 | */ 31 | public enum RequestType { 32 | 33 | // WARNING: reordering of request type will break protocol, ordinal values 34 | // are used for request type 35 | WRITE(() -> new WriteRequestPacket(), () -> new WriteResponsePacket()), 36 | TOPIC_METADATA(() -> new TopicMetadataRequestPacket(), 37 | () -> new TopicMetadataResponsePacket()), 38 | READ(() -> new ReadRequestPacket(), () -> new ReadResponsePacket()); 39 | 40 | public Supplier requestImplementationSupplier; 41 | public Supplier responseImplementationSupplier; 42 | 43 | private RequestType(Supplier requestImplementationSupplier, 44 | Supplier responseImplementationSupplier) { 45 | this.requestImplementationSupplier = requestImplementationSupplier; 46 | this.responseImplementationSupplier = responseImplementationSupplier; 47 | } 48 | 49 | public static final short PROTOCOL_VERSION = 3; 50 | 51 | public static RequestType extractPacketType(ByteBuf inBuffer) throws IOException { 52 | int requestTypeCode = (int) inBuffer.readByte(); 53 | RequestType[] values = RequestType.values(); 54 | if (requestTypeCode > values.length - 1) { 55 | throw new IOException("Invalid request type:" + requestTypeCode); 56 | } 57 | RequestType requestType = values[requestTypeCode];// request type 58 | return requestType; 59 | } 60 | 61 | } 62 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/protocol/ResponseCodes.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.protocol; 17 | 18 | public class ResponseCodes { 19 | 20 | public static final short OK = 200; 21 | public static final short UNAUTHORIZED = 401; 22 | public static final short SERVICE_UNAVAILABLE = 503; 23 | public static final short NOT_FOUND = 404; 24 | public static final short INTERNAL_SERVER_ERROR = 500; 25 | public static final short BAD_REQUEST = 400; 26 | public static final short REQUEST_FAILED = 502; 27 | public static final short REDIRECT = 302; 28 | public static final short NO_DATA = 204; 29 | 30 | private ResponseCodes() { 31 | } 32 | 33 | } 34 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/protocol/TopicAssignment.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.protocol; 17 | 18 | public class TopicAssignment extends TopicConfig { 19 | private long assignmentTimestamp = -1; 20 | private double assignmentInputTrafficMB; 21 | 22 | public TopicAssignment() { 23 | super(); 24 | } 25 | 26 | public TopicAssignment(TopicConfig topicConfig, double assignmentInputTrafficMB) { 27 | super(topicConfig); 28 | this.assignmentTimestamp = System.currentTimeMillis(); 29 | this.assignmentInputTrafficMB = assignmentInputTrafficMB; 30 | } 31 | 32 | public TopicAssignment(TopicConfig topicConfig, double assignmentInputTrafficMB, long assignmentTimestamp) { 33 | this(topicConfig, assignmentInputTrafficMB); 34 | this.assignmentTimestamp = assignmentTimestamp; 35 | } 36 | 37 | public long getAssignmentTimestamp() { 38 | return assignmentTimestamp; 39 | } 40 | 41 | public void setAssignmentTimestamp(long assignmentTimestamp) { 42 | this.assignmentTimestamp = assignmentTimestamp; 43 | } 44 | 45 | @Override 46 | public double getInputTrafficMB() { 47 | return assignmentInputTrafficMB; 48 | } 49 | 50 | @Override 51 | public void setInputTrafficMB(double inputTrafficMB) { 52 | assignmentInputTrafficMB = inputTrafficMB; 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/protocol/TopicMetadataRequestPacket.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.protocol; 17 | 18 | import java.io.IOException; 19 | 20 | import io.netty.buffer.ByteBuf; 21 | 22 | public class TopicMetadataRequestPacket implements Packet { 23 | 24 | private String topic; 25 | 26 | public TopicMetadataRequestPacket() { 27 | } 28 | 29 | public TopicMetadataRequestPacket(String topic) { 30 | this.topic = topic; 31 | } 32 | 33 | @Override 34 | public void readFields(ByteBuf buf, short protocolVersion) throws IOException { 35 | topic = ProtocolUtils.readStringWithTwoByteEncoding(buf); 36 | } 37 | 38 | @Override 39 | public void write(ByteBuf buf, short protocolVersion) { 40 | ProtocolUtils.writeStringWithTwoByteEncoding(buf, topic); 41 | } 42 | 43 | @Override 44 | public int getSize(short protocolVersion) { 45 | return ProtocolUtils.getStringSerializedSizeWithTwoByteEncoding(topic); 46 | } 47 | 48 | public String getTopic() { 49 | return topic; 50 | } 51 | 52 | } 53 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/protocol/TopicMetadataResponsePacket.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.protocol; 17 | 18 | import java.io.IOException; 19 | 20 | import io.netty.buffer.ByteBuf; 21 | 22 | public class TopicMetadataResponsePacket implements Packet { 23 | 24 | private TopicMetadata metadata; 25 | 26 | public TopicMetadataResponsePacket() { 27 | } 28 | 29 | public TopicMetadataResponsePacket(TopicMetadata metadata) { 30 | this.metadata = metadata; 31 | } 32 | 33 | @Override 34 | public void readFields(ByteBuf buf, short protocolVersion) throws IOException { 35 | metadata = new TopicMetadata(); 36 | metadata.readFields(buf, protocolVersion); 37 | } 38 | 39 | @Override 40 | public void write(ByteBuf buf, short protocolVersion) { 41 | metadata.write(buf, protocolVersion); 42 | } 43 | 44 | @Override 45 | public int getSize(short protocolVersion) { 46 | return metadata.getSize(protocolVersion); 47 | } 48 | 49 | public TopicMetadata getMetadata() { 50 | return metadata; 51 | } 52 | 53 | @Override 54 | public String toString() { 55 | return "TopicMetadataResponsePacket [metadata=" + metadata + "]"; 56 | } 57 | 58 | } 59 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/protocol/TransportPacket.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.protocol; 17 | 18 | public abstract class TransportPacket implements Packet { 19 | protected short protocolVersion; 20 | protected long clientRequestId; 21 | protected RequestType requestType; 22 | 23 | public TransportPacket() { 24 | 25 | } 26 | 27 | public TransportPacket(short protocolVersion, long clientRequestId, 28 | RequestType requestType) { 29 | this.protocolVersion = protocolVersion; 30 | this.clientRequestId = clientRequestId; 31 | this.requestType = requestType; 32 | } 33 | 34 | public short getProtocolVersion() { 35 | return protocolVersion; 36 | } 37 | 38 | public void setProtocolVersion(short protocolVersion) { 39 | this.protocolVersion = protocolVersion; 40 | } 41 | 42 | public long getClientRequestId() { 43 | return clientRequestId; 44 | } 45 | 46 | public void setClientRequestId(long clientRequestId) { 47 | this.clientRequestId = clientRequestId; 48 | } 49 | 50 | public RequestType getRequestType() { 51 | return requestType; 52 | } 53 | 54 | public void setRequestType(RequestType requestType) { 55 | this.requestType = requestType; 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/commons/protocol/WriteResponsePacket.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.protocol; 17 | 18 | import io.netty.buffer.ByteBuf; 19 | 20 | public class WriteResponsePacket implements Packet { 21 | 22 | @Override 23 | public void readFields(ByteBuf buf, short protocolVersion) { 24 | } 25 | 26 | @Override 27 | public void write(ByteBuf buf, short protocolVersion) { 28 | } 29 | 30 | @Override 31 | public int getSize(short protocolVersion) { 32 | return 0; 33 | } 34 | 35 | 36 | } -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/core/commons/MemqProcessingThreadFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.commons; 17 | 18 | import java.util.concurrent.ThreadFactory; 19 | import java.util.concurrent.atomic.AtomicInteger; 20 | 21 | public final class MemqProcessingThreadFactory implements ThreadFactory { 22 | private final String threadBaseName; 23 | private AtomicInteger counter = new AtomicInteger(); 24 | 25 | public MemqProcessingThreadFactory(String threadBaseName) { 26 | this.threadBaseName = threadBaseName; 27 | } 28 | 29 | @Override 30 | public Thread newThread(Runnable r) { 31 | Thread th = new Thread(r); 32 | th.setName(threadBaseName + counter.incrementAndGet()); 33 | th.setDaemon(true); 34 | return th; 35 | } 36 | } -------------------------------------------------------------------------------- /memq-commons/src/main/java/com/pinterest/memq/core/utils/DaemonThreadFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.utils; 17 | 18 | import java.util.concurrent.ThreadFactory; 19 | 20 | public class DaemonThreadFactory implements ThreadFactory { 21 | 22 | public static DaemonThreadFactory INSTANCE = new DaemonThreadFactory(); 23 | 24 | @Override 25 | public Thread newThread(Runnable r) { 26 | Thread th = new Thread(r); 27 | th.setDaemon(true); 28 | return th; 29 | } 30 | } -------------------------------------------------------------------------------- /memq-commons/src/main/thrift/memq.thrift: -------------------------------------------------------------------------------- 1 | namespace java com.pinterest.memq.commons 2 | 3 | struct LogRequest { 4 | 1: required string kafkaClusterSignature; 5 | 2: required list brokerLists; 6 | 3: required string acks = "1"; 7 | 4: optional string partitionerClass; 8 | 5: optional string keySerializerClass = "org.apache.kafka.common.serialization.ByteArraySerializer"; 9 | 6: optional string valueSerializerClass = "org.apache.kafka.common.serialization.ByteArraySerializer"; 10 | 7: optional string compressionType; 11 | 8: optional i32 maxRequestSize = 1000000; 12 | 9: optional bool sslEnabled = 0; 13 | 10: optional map sslSettings; 14 | 11: optional bool transactionEnabled = 0; 15 | 12: optional i32 transactionTimeoutMs = 6000; 16 | 13: optional i32 retries = 5; 17 | 14: optional i32 bufferMemory = 33554432; 18 | } -------------------------------------------------------------------------------- /memq-commons/src/test/java/com/pinterest/memq/commons/protocol/TestTopicMetadataPacket.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.commons.protocol; 17 | 18 | import static org.junit.Assert.assertEquals; 19 | 20 | import java.io.IOException; 21 | import java.util.HashSet; 22 | import java.util.Properties; 23 | 24 | import org.junit.Test; 25 | 26 | import com.pinterest.memq.commons.protocol.Broker.BrokerType; 27 | 28 | import io.netty.buffer.ByteBuf; 29 | import io.netty.buffer.PooledByteBufAllocator; 30 | 31 | public class TestTopicMetadataPacket { 32 | 33 | @Test 34 | public void testTopicMetadataRequest() throws IOException { 35 | TopicMetadataRequestPacket request = new TopicMetadataRequestPacket("test"); 36 | ByteBuf buf = PooledByteBufAllocator.DEFAULT.buffer(); 37 | request.write(buf, RequestType.PROTOCOL_VERSION); 38 | 39 | request = new TopicMetadataRequestPacket(); 40 | request.readFields(buf, RequestType.PROTOCOL_VERSION); 41 | assertEquals("test", request.getTopic()); 42 | } 43 | 44 | @Test 45 | public void testTopicMetadataResponse() throws IOException { 46 | Properties storageProperties = new Properties(); 47 | storageProperties.setProperty("prop1", "xyz"); 48 | storageProperties.setProperty("prop2", String.valueOf(212)); 49 | TopicMetadata md = new TopicMetadata("test23", "delayeddevnull", storageProperties); 50 | assertEquals(0, md.getWriteBrokers().size()); 51 | md.getWriteBrokers().add(new Broker("127.0.0.1", (short) 9092, "2xl", "us-east-1a", 52 | BrokerType.WRITE, new HashSet<>())); 53 | TopicMetadataResponsePacket response = new TopicMetadataResponsePacket(md); 54 | ByteBuf buf = PooledByteBufAllocator.DEFAULT.buffer(); 55 | response.write(buf, RequestType.PROTOCOL_VERSION); 56 | 57 | response = new TopicMetadataResponsePacket(); 58 | response.readFields(buf, RequestType.PROTOCOL_VERSION); 59 | TopicMetadata metadata = response.getMetadata(); 60 | assertEquals("test23", metadata.getTopicName()); 61 | assertEquals("delayeddevnull", metadata.getStorageHandlerName()); 62 | 63 | assertEquals(2, metadata.getStorageHandlerConfig().size()); 64 | assertEquals(1, metadata.getWriteBrokers().size()); 65 | 66 | md = new TopicMetadata("test23", "delayeddevnull", new Properties()); 67 | buf = PooledByteBufAllocator.DEFAULT.buffer(); 68 | response.write(buf, RequestType.PROTOCOL_VERSION); 69 | response = new TopicMetadataResponsePacket(); 70 | response.readFields(buf, RequestType.PROTOCOL_VERSION); 71 | metadata = response.getMetadata(); 72 | assertEquals("test23", metadata.getTopicName()); 73 | assertEquals("delayeddevnull", metadata.getStorageHandlerName()); 74 | } 75 | 76 | } 77 | -------------------------------------------------------------------------------- /memq-examples/.gitignore: -------------------------------------------------------------------------------- 1 | /target/ 2 | -------------------------------------------------------------------------------- /memq-examples/pom.xml: -------------------------------------------------------------------------------- 1 | 4 | 4.0.0 5 | 6 | com.pinterest.memq 7 | memq-parent 8 | 0.2.22-SNAPSHOT 9 | ../pom.xml 10 | 11 | memq-examples 12 | 13 | 14 | com.pinterest.memq 15 | memq-client 16 | ${project.version} 17 | 18 | 19 | org.slf4j 20 | slf4j-log4j12 21 | 1.7.25 22 | 23 | 24 | commons-lang 25 | commons-lang 26 | 2.6 27 | 28 | 29 | -------------------------------------------------------------------------------- /memq-python-client/README.md: -------------------------------------------------------------------------------- 1 | # MemQ Python Client 2 | A native Python client for MemQ 3 | 4 | ## License 5 | 6 | Apache 2.0 7 | 8 | ## Support Matrix 9 | 10 | Currently the client is in early development and only supports: 11 | - consumer 12 | - batch (non-streaming) topic reads 13 | 14 | ## Build 15 | 16 | *Run build* 17 | 18 | ``` 19 | python3 setup.py build 20 | ``` 21 | 22 | *Run package* 23 | 24 | ``` 25 | python3 setup.py sdist 26 | ``` 27 | 28 | *Run install* 29 | 30 | ``` 31 | python3 setup.py install 32 | ``` 33 | 34 | *Run tests* 35 | 36 | ``` 37 | python3 setup.py test 38 | ``` -------------------------------------------------------------------------------- /memq-python-client/pyproject.toml: -------------------------------------------------------------------------------- 1 | [build-system] 2 | requires = ["setuptools", "wheel"] 3 | build-backend = "setuptools.build_meta" 4 | -------------------------------------------------------------------------------- /memq-python-client/requirements.txt: -------------------------------------------------------------------------------- 1 | boto3==1.16.14 2 | zstandard==0.14.0 -------------------------------------------------------------------------------- /memq-python-client/run_tests.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | set -o verbose 4 | 5 | WD=$(pwd) 6 | echo "current working directory: $WD" 7 | if [ -z "$PYTHONPATH" ]; then 8 | PYTHONPATH=. 9 | else 10 | PYTHONPATH=$PYTHONPATH:. 11 | fi 12 | echo "using PYTHONPATH: $PYTHONPATH" 13 | 14 | python3 setup.py test -------------------------------------------------------------------------------- /memq-python-client/setup.cfg: -------------------------------------------------------------------------------- 1 | [metadata] 2 | # This includes the license file(s) in the wheel. 3 | # https://wheel.readthedocs.io/en/stable/user_guide.html#including-license-files-in-the-generated-wheel-file 4 | license_files = LICENSE -------------------------------------------------------------------------------- /memq-python-client/setup.py: -------------------------------------------------------------------------------- 1 | """MemQ setuptools 2 | See: 3 | https://packaging.python.org/guides/distributing-packages-using-setuptools/ 4 | """ 5 | 6 | # Always prefer setuptools over distutils 7 | from setuptools import setup, find_packages 8 | import pathlib 9 | 10 | here = pathlib.Path(__file__).parent.resolve() 11 | 12 | # Get the long description from the README file 13 | long_description = (here / 'README.md').read_text(encoding='utf-8') 14 | 15 | # Arguments marked as "Required" below must be included for upload to PyPI. 16 | # Fields marked as "Optional" may be commented out. 17 | 18 | setup( 19 | name='memq-client', # Required 20 | version='0.2.22-SNAPSHOT', # Required 21 | description='MemQ Python Client', # Optional 22 | long_description=long_description, # Optional 23 | long_description_content_type='text/markdown', # Optional (see note above) 24 | author='Ambud Sharma', # Optional 25 | author_email='logging@pinterest.com', # Optional 26 | classifiers=[ # Optional 27 | 'Development Status :: 3 - Alpha', 28 | 'Intended Audience :: Developers', 29 | 'Topic :: Software Development :: Build Tools', 30 | 'License :: OSI Approved :: Apache 2.0', 31 | 'Programming Language :: Python :: 3', 32 | 'Programming Language :: Python :: 3.6', 33 | 'Programming Language :: Python :: 3.7', 34 | 'Programming Language :: Python :: 3.8', 35 | 'Programming Language :: Python :: 3.9', 36 | 'Programming Language :: Python :: 3 :: Only', 37 | ], 38 | keywords='memq-client', # Optional 39 | package_dir={'': 'src'}, # Optional 40 | packages=find_packages(where='src'), # Required 41 | python_requires='>=3.6, <4', 42 | install_requires=['zstandard', 'boto3'], # Optional 43 | extras_require={ # Optional 44 | 'test': ['coverage'], 45 | }, 46 | entry_points={ # Optional 47 | 'console_scripts': [ 48 | 'sample=memq.example_batch_consumer:main', 49 | ], 50 | }, 51 | 52 | project_urls={ # Optional 53 | 'Bug Reports': '#logging-incidents', 54 | }, 55 | ) -------------------------------------------------------------------------------- /memq-python-client/src/memq/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/memq/83f97daaef49ee14394d9227444c2f140df64802/memq-python-client/src/memq/__init__.py -------------------------------------------------------------------------------- /memq-python-client/src/memq/example_batch_consumer.py: -------------------------------------------------------------------------------- 1 | from memq.memqbatchconsumer import MemqBatchConsumerV1 2 | 3 | class ExampleBatchConsumer: 4 | 5 | def run(self, cluster, bucket, topic): 6 | consumer = MemqBatchConsumerV1(cluster=cluster, bucket=bucket, topic=topic) 7 | consumer.fetch_objects() 8 | while consumer.has_next(): 9 | itr = consumer.poll() 10 | while itr.has_next(): 11 | message = itr.next() 12 | print(str(len(message.value)) + " headers:" + str(message.write_timestamp)) 13 | 14 | if __name__ == "__main__": 15 | import sys;sys.argv = [1:] 16 | ExampleBatchConsumer.run() 17 | -------------------------------------------------------------------------------- /memq-python-client/src/memq/inputstream.py: -------------------------------------------------------------------------------- 1 | ''' 2 | Created on Feb 2, 2021 3 | 4 | @author: ambudsharma 5 | ''' 6 | 7 | import struct 8 | 9 | class InputStream: 10 | def __init__(self, stream): 11 | self.stream = stream 12 | 13 | def read(self, count): 14 | return self.stream.read(count) 15 | 16 | def available(self): 17 | return self.stream.getbuffer().nbytes - self.stream.tell() 18 | 19 | class DataInputStream(InputStream): 20 | def __init__(self, stream): 21 | super(DataInputStream, self).__init__(stream) 22 | 23 | def read_boolean(self): 24 | return struct.unpack('?', self.stream.read(1))[0] 25 | 26 | def read_byte(self): 27 | return struct.unpack('b', self.stream.read(1))[0] 28 | 29 | def read_unsigned_byte(self): 30 | return struct.unpack('B', self.stream.read(1))[0] 31 | 32 | def read_char(self): 33 | return chr(struct.unpack('>H', self.stream.read(2))[0]) 34 | 35 | def read_double(self): 36 | return struct.unpack('>d', self.stream.read(8))[0] 37 | 38 | def read_float(self): 39 | return struct.unpack('>f', self.stream.read(4))[0] 40 | 41 | def read_short(self): 42 | return struct.unpack('>h', self.stream.read(2))[0] 43 | 44 | def read_unsigned_short(self): 45 | return struct.unpack('>H', self.stream.read(2))[0] 46 | 47 | def read_long(self): 48 | return struct.unpack('>q', self.stream.read(8))[0] 49 | 50 | def read_utf(self): 51 | utf_length = struct.unpack('>H', self.stream.read(2))[0] 52 | return self.stream.read(utf_length) 53 | 54 | def read_int(self): 55 | return struct.unpack('>i', self.stream.read(4))[0] -------------------------------------------------------------------------------- /memq-python-client/src/memq/kafkanotificationsource.py: -------------------------------------------------------------------------------- 1 | ''' 2 | Created on Feb 3, 2021 3 | 4 | @author: ambudsharma 5 | ''' 6 | 7 | class KafkaNotificationSource(object): 8 | ''' 9 | classdocs 10 | ''' 11 | 12 | 13 | def __init__(self, params): 14 | ''' 15 | Constructor 16 | ''' 17 | -------------------------------------------------------------------------------- /memq-python-client/src/memq/memqbatchconsumer.py: -------------------------------------------------------------------------------- 1 | ''' 2 | Created on Feb 3, 2021 3 | 4 | @author: ambudsharma 5 | ''' 6 | 7 | import boto3 8 | from memq.memqlogmessageiterator import MemqLogMessageIterator, Deserializer, \ 9 | MemqLogMessage 10 | from memq.inputstream import DataInputStream 11 | 12 | 13 | class MemqBatchConsumerV1(object): 14 | ''' 15 | classdocs 16 | ''' 17 | # generate static shard path combinations 18 | bucket_shard_paths = [] 19 | for i in range(16): 20 | for j in range(16): 21 | bucket_shard_paths.append(format(i, 'x') + format(j, 'x')) 22 | 23 | def __init__(self, cluster, bucket, topic): 24 | ''' 25 | Constructor 26 | ''' 27 | self.cluster = cluster 28 | self.bucket = bucket 29 | self.topic = topic 30 | 31 | self.s3 = boto3.client('s3') 32 | 33 | def fetch_objects(self): 34 | i = '00' 35 | response = self.s3.list_objects(Bucket=self.bucket, Prefix=i + "/" + self.cluster + "/topics/" + self.topic) 36 | if response['ResponseMetadata'] == None or response['ResponseMetadata']['HTTPStatusCode'] != 200: 37 | raise Exception("Failed request:" + response) 38 | 39 | objects = response['Contents'] 40 | self.keys = [] 41 | self.index = 0 42 | for o in objects: 43 | self.keys.append(o['Key']) 44 | # for i in self.bucket_shard_paths: 45 | # print("s3://" + bucket + "/" + i + "/" + cluster + "/topics/" + topic) 46 | 47 | def has_next(self) -> bool: 48 | return self.index < self.keys.__len__() 49 | 50 | def poll(self) -> MemqLogMessageIterator: 51 | key = self.keys[self.index] 52 | print("Fetching:" + key) 53 | response = self.s3.get_object(Bucket=self.bucket, Key=key) 54 | self.index += 1 55 | stream = DataInputStream(response['Body']) 56 | curr_notification_obj = { 57 | MemqLogMessage.INTERNAL_FIELD_TOPIC:"test", 58 | MemqLogMessage.INTERNAL_FIELD_OBJECT_SIZE:response['ContentLength'], 59 | MemqLogMessage.INTERNAL_FIELD_NOTIFICATION_PARTITION_ID:0, 60 | MemqLogMessage.INTERNAL_FIELD_NOTIFICATION_READ_TIMESTAMP:0, 61 | MemqLogMessage.INTERNAL_FIELD_NOTIFICATION_PARTITION_OFFSET:0 62 | } 63 | return MemqLogMessageIterator(self.cluster, stream, curr_notification_obj, Deserializer(), Deserializer()) 64 | -------------------------------------------------------------------------------- /memq-python-client/src/memq/memqconsumer.py: -------------------------------------------------------------------------------- 1 | ''' 2 | Created on Feb 3, 2021 3 | 4 | @author: ambudsharma 5 | ''' 6 | 7 | class MemqConsumer(object): 8 | ''' 9 | classdocs 10 | ''' 11 | 12 | 13 | def __init__(self, params): 14 | ''' 15 | Constructor 16 | ''' 17 | -------------------------------------------------------------------------------- /memq-python-client/tests/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/memq/83f97daaef49ee14394d9227444c2f140df64802/memq-python-client/tests/__init__.py -------------------------------------------------------------------------------- /memq-python-client/tox.ini: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed under the Apache License, Version 2.0 (the "License"); 3 | # you may not use this file except in compliance with the License. 4 | # You may obtain a copy of the License at 5 | # 6 | # http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software 9 | # distributed under the License is distributed on an "AS IS" BASIS, 10 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | # See the License for the specific language governing permissions and 12 | # limitations under the License. 13 | 14 | [tox] 15 | 16 | envlist=py36 17 | skipsdist=True 18 | 19 | [testenv] 20 | 21 | basepython= 22 | py36: python3.6 23 | 24 | commands= 25 | pip install -e .[develop] 26 | {toxinidir}/run_tests.sh 27 | -------------------------------------------------------------------------------- /memq.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pinterest/memq/83f97daaef49ee14394d9227444c2f140df64802/memq.png -------------------------------------------------------------------------------- /memq/.gitignore: -------------------------------------------------------------------------------- 1 | /target/ 2 | -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/clustering/BalanceStrategy.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.clustering; 17 | 18 | import java.util.Set; 19 | 20 | import com.pinterest.memq.commons.protocol.Broker; 21 | import com.pinterest.memq.commons.protocol.TopicConfig; 22 | import com.pinterest.memq.core.config.MemqConfig; 23 | 24 | public abstract class BalanceStrategy { 25 | protected final MemqConfig memqConfig; 26 | 27 | public abstract Set balance(Set topics, Set brokers); 28 | 29 | public BalanceStrategy() { 30 | this.memqConfig = null; 31 | } 32 | 33 | public BalanceStrategy(MemqConfig memqConfig) { 34 | this.memqConfig = memqConfig; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/clustering/ExpirationPartitionBalanceStrategyWithAssignmentFreeze.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.memq.core.clustering; 2 | 3 | import com.pinterest.memq.commons.protocol.Broker; 4 | import com.pinterest.memq.commons.protocol.TopicAssignment; 5 | import com.pinterest.memq.commons.protocol.TopicConfig; 6 | import com.pinterest.memq.core.config.MemqConfig; 7 | 8 | import java.util.ArrayList; 9 | import java.util.HashSet; 10 | import java.util.List; 11 | import java.util.Set; 12 | import java.util.logging.Logger; 13 | 14 | public class ExpirationPartitionBalanceStrategyWithAssignmentFreeze extends ExpirationPartitionBalanceStrategyWithErrorHandling { 15 | 16 | private static final Logger logger = 17 | Logger.getLogger(ExpirationPartitionBalanceStrategyWithAssignmentFreeze.class.getName()); 18 | 19 | public ExpirationPartitionBalanceStrategyWithAssignmentFreeze(MemqConfig memqConfig) { 20 | super(memqConfig); 21 | } 22 | 23 | /** 24 | * Use the existing assignment and send alert. 25 | * Refresher the topic assignment timestamp. 26 | * @param topics 27 | * @param brokers 28 | * @return brokers 29 | */ 30 | @Override 31 | protected Set handleBalancerError(Set topics, Set brokers) { 32 | logger.info("Trigger assignment freeze and send alert. Current assignment: " + brokers); 33 | this.sendAlert(); 34 | List brokerList = new ArrayList<>(brokers); 35 | for (Broker broker : brokerList) { 36 | for (TopicAssignment topicAssignment : broker.getAssignedTopics()) { 37 | topicAssignment.setAssignmentTimestamp(System.currentTimeMillis()); 38 | } 39 | } 40 | return new HashSet<>(brokers); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/config/AuthorizerConfig.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.config; 17 | 18 | import java.util.Map; 19 | 20 | public class AuthorizerConfig { 21 | 22 | private String authorizerClass; 23 | private Map configs; 24 | 25 | public String getAuthorizerClass() { 26 | return authorizerClass; 27 | } 28 | 29 | public void setAuthorizerClass(String authorizerClass) { 30 | this.authorizerClass = authorizerClass; 31 | } 32 | 33 | public Map getConfigs() { 34 | return configs; 35 | } 36 | 37 | public void setConfigs(Map configs) { 38 | this.configs = configs; 39 | } 40 | 41 | } 42 | -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/config/ClusteringConfig.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.config; 17 | 18 | public class ClusteringConfig { 19 | 20 | private String zookeeperConnectionString; 21 | private int batchMultiplier = 3; 22 | private boolean enableLeaderSelector = true; 23 | private boolean enableBalancer = true; 24 | private boolean enableLocalAssigner = true; 25 | private boolean addBootstrapTopics = true; 26 | private boolean enableExpiration = true; 27 | 28 | public boolean isAddBootstrapTopics() { 29 | return addBootstrapTopics; 30 | } 31 | 32 | public void setAddBootstrapTopics(boolean addBootstrapTopics) { 33 | this.addBootstrapTopics = addBootstrapTopics; 34 | } 35 | 36 | public String getZookeeperConnectionString() { 37 | return zookeeperConnectionString; 38 | } 39 | 40 | public void setZookeeperConnectionString(String zookeeperConnectionString) { 41 | this.zookeeperConnectionString = zookeeperConnectionString; 42 | } 43 | 44 | public int getBatchMultiplier() { 45 | return batchMultiplier; 46 | } 47 | 48 | public void setBatchMultiplier(int batchMultiplier) { 49 | this.batchMultiplier = batchMultiplier; 50 | } 51 | 52 | public boolean isEnableLeaderSelector() { 53 | return enableLeaderSelector; 54 | } 55 | 56 | public void setEnableLeaderSelector(boolean enableLeaderSelector) { 57 | this.enableLeaderSelector = enableLeaderSelector; 58 | } 59 | 60 | public boolean isEnableBalancer() { 61 | return enableBalancer; 62 | } 63 | 64 | public void setEnableBalancer(boolean enableBalancer) { 65 | this.enableBalancer = enableBalancer; 66 | } 67 | 68 | public boolean isEnableLocalAssigner() { 69 | return enableLocalAssigner; 70 | } 71 | 72 | public void setEnableLocalAssigner(boolean enableLocalAssigner) { 73 | this.enableLocalAssigner = enableLocalAssigner; 74 | } 75 | 76 | public boolean isEnableExpiration() { 77 | return enableExpiration; 78 | } 79 | 80 | public void setEnableExpiration(boolean enableExpiration) { 81 | this.enableExpiration = enableExpiration; 82 | } 83 | } 84 | -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/config/EC2EnvironmentProvider.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.config; 17 | 18 | import software.amazon.awssdk.regions.internal.util.EC2MetadataUtils; 19 | 20 | public class EC2EnvironmentProvider extends EnvironmentProvider { 21 | 22 | public EC2EnvironmentProvider() { 23 | } 24 | 25 | @Override 26 | public String getRack() { 27 | return EC2MetadataUtils.getAvailabilityZone(); 28 | } 29 | 30 | @Override 31 | public String getIP() { 32 | return EC2MetadataUtils.getInstanceInfo().getPrivateIp(); 33 | } 34 | 35 | @Override 36 | public String getInstanceType() { 37 | return EC2MetadataUtils.getInstanceType(); 38 | } 39 | 40 | } 41 | -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/config/EnvironmentProvider.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.config; 17 | 18 | public abstract class EnvironmentProvider { 19 | 20 | public abstract String getRack(); 21 | 22 | public abstract String getIP(); 23 | 24 | public abstract String getInstanceType(); 25 | 26 | } 27 | -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/config/LocalEnvironmentProvider.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.memq.core.config; 2 | 3 | public class LocalEnvironmentProvider extends EnvironmentProvider { 4 | 5 | @Override 6 | public String getRack() { 7 | return "local"; 8 | } 9 | 10 | @Override 11 | public String getInstanceType() { 12 | return "2xl"; 13 | } 14 | 15 | @Override 16 | public String getIP() { 17 | return "127.0.0.1"; 18 | } 19 | 20 | } 21 | -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/config/OpenTsdbConfiguration.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.config; 17 | 18 | public class OpenTsdbConfiguration { 19 | 20 | private String host; 21 | private int port; 22 | private int frequencyInSeconds; 23 | 24 | /** 25 | * @return the host 26 | */ 27 | public String getHost() { 28 | return host; 29 | } 30 | /** 31 | * @param host the host to set 32 | */ 33 | public void setHost(String host) { 34 | this.host = host; 35 | } 36 | /** 37 | * @return the port 38 | */ 39 | public int getPort() { 40 | return port; 41 | } 42 | /** 43 | * @param port the port to set 44 | */ 45 | public void setPort(int port) { 46 | this.port = port; 47 | } 48 | /** 49 | * @return the frequencyInSeconds 50 | */ 51 | public int getFrequencyInSeconds() { 52 | return frequencyInSeconds; 53 | } 54 | /** 55 | * @param frequencyInSeconds the frequencyInSeconds to set 56 | */ 57 | public void setFrequencyInSeconds(int frequencyInSeconds) { 58 | this.frequencyInSeconds = frequencyInSeconds; 59 | } 60 | 61 | } -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/mon/MemqMgrHealthCheck.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.mon; 17 | 18 | import com.codahale.metrics.health.HealthCheck; 19 | import com.pinterest.memq.core.MemqManager; 20 | 21 | public class MemqMgrHealthCheck extends HealthCheck { 22 | 23 | private MemqManager mgr; 24 | 25 | public MemqMgrHealthCheck(MemqManager mgr) { 26 | this.mgr = mgr; 27 | } 28 | 29 | @Override 30 | protected Result check() throws Exception { 31 | if (mgr.isRunning()) { 32 | return Result.healthy(); 33 | } else { 34 | return Result.unhealthy("Memq Manager is not healthy"); 35 | } 36 | } 37 | 38 | } 39 | -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/processing/Ackable.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.processing; 17 | 18 | public interface Ackable { 19 | 20 | void markProcessed(long ackId); 21 | 22 | void markPending(long ackId); 23 | 24 | void markFailed(long ackId); 25 | 26 | } 27 | -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/processing/ProcessingStatus.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.processing; 17 | 18 | public enum ProcessingStatus { 19 | 20 | FAILED, 21 | PENDING, 22 | PROCESSED 23 | 24 | } 25 | -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/processing/TopicProcessorState.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.processing; 17 | 18 | public enum TopicProcessorState { 19 | 20 | INITIALIZING, 21 | RUNNING, 22 | STOPPING, 23 | STOPPED 24 | 25 | } 26 | -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/rpc/BrokerTrafficShapingHandler.java: -------------------------------------------------------------------------------- 1 | package com.pinterest.memq.core.rpc; 2 | 3 | import com.codahale.metrics.MetricRegistry; 4 | import io.netty.handler.traffic.GlobalTrafficShapingHandler; 5 | 6 | import java.util.concurrent.ScheduledExecutorService; 7 | import java.util.concurrent.TimeUnit; 8 | import java.util.logging.Logger; 9 | 10 | public class BrokerTrafficShapingHandler extends GlobalTrafficShapingHandler { 11 | 12 | public static final String READ_LIMIT_METRIC_NAME = "broker.traffic.read.limit"; 13 | private static final Logger logger = Logger.getLogger(BrokerTrafficShapingHandler.class.getName()); 14 | private int metricsReportingIntervalSec = 60; // default 1 minute 15 | private final MetricRegistry registry; 16 | 17 | public BrokerTrafficShapingHandler(ScheduledExecutorService executor, 18 | long writeLimit, 19 | long readLimit, 20 | long checkInterval, 21 | MetricRegistry registry) { 22 | super(executor, writeLimit, readLimit, checkInterval); 23 | this.registry = registry; 24 | } 25 | 26 | /** 27 | * Set the interval for metrics reporting. 28 | * If the interval is less than or equal to 0, metrics reporting is disabled. 29 | * @param intervalSec 30 | */ 31 | public void setMetricsReportingIntervalSec(int intervalSec) { 32 | metricsReportingIntervalSec = intervalSec; 33 | } 34 | 35 | /** 36 | * Get the interval for metrics reporting. 37 | * @return intervalSec 38 | */ 39 | public int getMetricsReportingIntervalSec() { 40 | return metricsReportingIntervalSec; 41 | } 42 | 43 | /** 44 | * Start periodic metrics reporting. The interval is specified by metricsReportingIntervalSec. 45 | * If the interval is less than or equal to 0, metrics reporting is disabled. 46 | * Overriding channel methods to send metrics can cause performance issues. 47 | * So we choose to send metrics in a separate thread periodically. 48 | * @param executorService 49 | */ 50 | public void startPeriodicMetricsReporting(ScheduledExecutorService executorService) { 51 | if (metricsReportingIntervalSec <= 0) { 52 | logger.warning("Metrics reporting is disabled because the interval is less than or equal to 0."); 53 | return; 54 | } 55 | logger.info(String.format("Starting periodic metrics reporting every %d seconds.", 56 | metricsReportingIntervalSec)); 57 | Runnable reportTask = this::reportMetrics; 58 | executorService.scheduleAtFixedRate( 59 | reportTask, 0, metricsReportingIntervalSec, TimeUnit.SECONDS); 60 | } 61 | 62 | /** 63 | * Report read limit metric to the registry. 64 | */ 65 | public void reportMetrics() { 66 | long readLimit = this.getReadLimit(); 67 | registry.gauge(READ_LIMIT_METRIC_NAME, () -> () -> readLimit); 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/rpc/MemqProtocolFeatureMatrix.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.rpc; 17 | 18 | public class MemqProtocolFeatureMatrix { 19 | 20 | public static final short MIN_PROTO_VERSION_CHECKSUM = 2; 21 | 22 | } 23 | -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/rpc/Response.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.rpc; 17 | 18 | import com.pinterest.memq.commons.protocol.RequestType; 19 | 20 | public abstract class Response { 21 | 22 | protected long clientRequestId; 23 | protected short responseCode; 24 | 25 | public Response(long clientRequestId, short responseCode) { 26 | this.clientRequestId = clientRequestId; 27 | this.responseCode = responseCode; 28 | } 29 | 30 | public abstract RequestType getRequestType(); 31 | 32 | public long getClientRequestId() { 33 | return clientRequestId; 34 | } 35 | 36 | public void setClientRequestId(long clientRequestId) { 37 | this.clientRequestId = clientRequestId; 38 | } 39 | 40 | public short getResponseCode() { 41 | return responseCode; 42 | } 43 | 44 | public void setResponseCode(short responseCode) { 45 | this.responseCode = responseCode; 46 | } 47 | 48 | public abstract byte[] getSerializedBytes(); 49 | 50 | } -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/rpc/WriteResponse.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.rpc; 17 | 18 | import com.pinterest.memq.commons.protocol.RequestType; 19 | 20 | public class WriteResponse extends Response { 21 | 22 | public WriteResponse(long requestId, short responseCode) { 23 | super(requestId, responseCode); 24 | } 25 | 26 | @Override 27 | public RequestType getRequestType() { 28 | return RequestType.WRITE; 29 | } 30 | 31 | @Override 32 | public byte[] getSerializedBytes() { 33 | return null; 34 | } 35 | 36 | } -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/security/Authorizer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.security; 17 | 18 | import java.security.Principal; 19 | 20 | import com.pinterest.memq.commons.protocol.RequestType; 21 | import com.pinterest.memq.core.config.AuthorizerConfig; 22 | 23 | public interface Authorizer { 24 | 25 | public void init(AuthorizerConfig config) throws Exception; 26 | 27 | public boolean authorize(Principal principal, 28 | String locus, 29 | String resource, 30 | RequestType operation); 31 | 32 | } 33 | -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/tools/TopicAdmin.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.tools; 17 | 18 | import java.io.File; 19 | import java.nio.file.Files; 20 | import java.util.concurrent.TimeUnit; 21 | 22 | import org.apache.commons.cli.CommandLine; 23 | import org.apache.commons.cli.CommandLineParser; 24 | import org.apache.commons.cli.DefaultParser; 25 | import org.apache.commons.cli.HelpFormatter; 26 | import org.apache.commons.cli.Option; 27 | import org.apache.commons.cli.Options; 28 | import org.apache.curator.RetryPolicy; 29 | import org.apache.curator.framework.CuratorFramework; 30 | import org.apache.curator.framework.CuratorFrameworkFactory; 31 | import org.apache.curator.retry.ExponentialBackoffRetry; 32 | 33 | import com.google.gson.Gson; 34 | import com.pinterest.memq.commons.protocol.TopicConfig; 35 | import com.pinterest.memq.core.clustering.MemqGovernor; 36 | 37 | public class TopicAdmin { 38 | 39 | private static final String ZOOKEEPER_CONNECTION = "zk"; 40 | private static final String TOPIC_CONFIG_JSON_FILE = "tcjf"; 41 | 42 | public static void main(String[] args) throws Exception { 43 | // create Options object 44 | Options options = new Options(); 45 | 46 | // add t option 47 | 48 | options.addOption(Option.builder().required(true).hasArg().longOpt(ZOOKEEPER_CONNECTION) 49 | .argName("zookeeper connection string").build()); 50 | options.addOption(Option.builder().required(true).hasArg().longOpt(TOPIC_CONFIG_JSON_FILE).argName("topic configuration json file").build()); 51 | 52 | CommandLineParser parser = new DefaultParser(); 53 | CommandLine cmd = null; 54 | 55 | try { 56 | cmd = parser.parse(options, args); 57 | } catch (Exception e) { 58 | printHelp(options); 59 | return; 60 | } 61 | 62 | String zookeeperConnectionString = cmd.getOptionValue(ZOOKEEPER_CONNECTION); 63 | String topicConfigFile = cmd.getOptionValue(TOPIC_CONFIG_JSON_FILE); 64 | RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3); 65 | CuratorFramework client = CuratorFrameworkFactory.newClient(zookeeperConnectionString, 66 | retryPolicy); 67 | client.start(); 68 | client.blockUntilConnected(100, TimeUnit.SECONDS); 69 | Gson gson = new Gson(); 70 | TopicConfig config = gson.fromJson( 71 | new String(Files.readAllBytes(new File(topicConfigFile).toPath())), TopicConfig.class); 72 | MemqGovernor.createTopic(client, config); 73 | client.close(); 74 | } 75 | 76 | private static void printHelp(Options options) { 77 | HelpFormatter formatter = new HelpFormatter(); 78 | formatter.printHelp("memqcli", options); 79 | } 80 | 81 | } 82 | -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/utils/CoreUtils.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.utils; 17 | 18 | import java.io.IOException; 19 | import java.util.List; 20 | 21 | import com.pinterest.memq.core.commons.Message; 22 | 23 | public class CoreUtils { 24 | 25 | private CoreUtils() { 26 | } 27 | 28 | public static long makeAckKey(Message message) { 29 | return makeAckKey(message.getServerRequestId(), message.getClientRequestId()); 30 | } 31 | 32 | public static long makeAckKey(long serverRequestId, long clientRequestId) { 33 | return serverRequestId * clientRequestId; 34 | } 35 | 36 | public static int batchSizeInBytes(List batch) { 37 | return batch.stream().mapToInt(b -> b.getBuf().writerIndex()).sum(); 38 | } 39 | 40 | public static int batchChecksum(List messageBatchAsList) throws IOException { 41 | CrcProcessor proc = new CrcProcessor(); 42 | for (Message message : messageBatchAsList) { 43 | message.getBuf().forEachByte(proc); 44 | } 45 | return proc.getChecksum(); 46 | } 47 | 48 | } -------------------------------------------------------------------------------- /memq/src/main/java/com/pinterest/memq/core/utils/CrcProcessor.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.utils; 17 | 18 | import io.netty.util.ByteProcessor; 19 | 20 | import java.util.zip.CRC32; 21 | 22 | public final class CrcProcessor implements ByteProcessor { 23 | 24 | private CRC32 crc32 = new CRC32(); 25 | 26 | @Override 27 | public boolean process(byte value) throws Exception { 28 | crc32.update(value); 29 | return true; 30 | } 31 | 32 | public int getChecksum() { 33 | return (int) crc32.getValue(); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /memq/src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.logger.httpclient.wire.header=WARN 2 | log4j.logger.httpclient.wire.content=WARN 3 | -------------------------------------------------------------------------------- /memq/src/main/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | -------------------------------------------------------------------------------- /memq/src/main/resources/logo.txt: -------------------------------------------------------------------------------- 1 | 2 | MMMMMMMM MMMMMMMMEEEEEEEEEEEEEEEEEEEEEEMMMMMMMM MMMMMMMM QQQQQQQQQ 3 | M:::::::M M:::::::ME::::::::::::::::::::EM:::::::M M:::::::M QQ:::::::::QQ 4 | M::::::::M M::::::::ME::::::::::::::::::::EM::::::::M M::::::::M QQ:::::::::::::QQ 5 | M:::::::::M M:::::::::MEE::::::EEEEEEEEE::::EM:::::::::M M:::::::::MQ:::::::QQQ:::::::Q 6 | M::::::::::M M::::::::::M E:::::E EEEEEEM::::::::::M M::::::::::MQ::::::O Q::::::Q 7 | M:::::::::::M M:::::::::::M E:::::E M:::::::::::M M:::::::::::MQ:::::O Q:::::Q 8 | M:::::::M::::M M::::M:::::::M E::::::EEEEEEEEEE M:::::::M::::M M::::M:::::::MQ:::::O Q:::::Q 9 | M::::::M M::::M M::::M M::::::M E:::::::::::::::E M::::::M M::::M M::::M M::::::MQ:::::O Q:::::Q 10 | M::::::M M::::M::::M M::::::M E:::::::::::::::E M::::::M M::::M::::M M::::::MQ:::::O Q:::::Q 11 | M::::::M M:::::::M M::::::M E::::::EEEEEEEEEE M::::::M M:::::::M M::::::MQ:::::O Q:::::Q 12 | M::::::M M:::::M M::::::M E:::::E M::::::M M:::::M M::::::MQ:::::O QQQQ:::::Q 13 | M::::::M MMMMM M::::::M E:::::E EEEEEEM::::::M MMMMM M::::::MQ::::::O Q::::::::Q 14 | M::::::M M::::::MEE::::::EEEEEEEE:::::EM::::::M M::::::MQ:::::::QQ::::::::Q 15 | M::::::M M::::::ME::::::::::::::::::::EM::::::M M::::::M QQ::::::::::::::Q 16 | M::::::M M::::::ME::::::::::::::::::::EM::::::M M::::::M QQ:::::::::::Q 17 | MMMMMMMM MMMMMMMMEEEEEEEEEEEEEEEEEEEEEEMMMMMMMM MMMMMMMM QQQQQQQQ::::QQ 18 | Q:::::Q 19 | -------------------------------------------------------------------------------- /memq/src/test/java/com/pinterest/memq/core/TestMemqManager.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core; 17 | 18 | import static org.junit.Assert.assertEquals; 19 | 20 | import java.io.File; 21 | import java.nio.file.Paths; 22 | import java.util.HashMap; 23 | 24 | import org.junit.Test; 25 | 26 | import com.pinterest.memq.commons.protocol.TopicAssignment; 27 | import com.pinterest.memq.commons.protocol.TopicConfig; 28 | import com.pinterest.memq.core.config.MemqConfig; 29 | 30 | public class TestMemqManager { 31 | 32 | @Test 33 | public void testTopicCache() throws Exception { 34 | new File("target/testmgrcache").delete(); 35 | MemqConfig configuration = new MemqConfig(); 36 | configuration.setTopicCacheFile("target/testmgrcache"); 37 | MemqManager mgr = new MemqManager(null, configuration, new HashMap<>()); 38 | mgr.init(); 39 | TopicConfig topicConfig = new TopicConfig(0, 1024, 100, "test", 10, 10, 2); 40 | TopicAssignment topicAssignment = new TopicAssignment(topicConfig, 10); 41 | topicAssignment.setStorageHandlerName("delayeddevnull"); 42 | mgr.createTopicProcessor(topicAssignment); 43 | mgr.updateTopicCache(); 44 | 45 | mgr = new MemqManager(null, configuration, new HashMap<>()); 46 | mgr.init(); 47 | assertEquals(1, mgr.getProcessorMap().size()); 48 | assertEquals(1, mgr.getTopicAssignment().size()); 49 | } 50 | 51 | @Test 52 | public void testTopicConfig() throws Exception { 53 | MemqConfig config = new MemqConfig(); 54 | File tmpFile = File.createTempFile("test", "", Paths.get("/tmp").toFile()); 55 | tmpFile.deleteOnExit(); 56 | config.setTopicCacheFile(tmpFile.toString()); 57 | MemqManager mgr = new MemqManager(null, new MemqConfig(), new HashMap<>()); 58 | TopicConfig topicConfig = new TopicConfig("test", "delayeddevnull"); 59 | TopicAssignment topicAssignment = new TopicAssignment(topicConfig, -1); 60 | mgr.createTopicProcessor(topicAssignment); 61 | 62 | long size = mgr.getTopicAssignment().iterator().next().getBatchSizeBytes(); 63 | topicAssignment = new TopicAssignment(new TopicConfig("test", "delayeddevnull"), -1); 64 | topicAssignment.setBatchSizeBytes(size + 100); 65 | mgr.updateTopic(topicAssignment); 66 | assertEquals(size + 100, mgr.getTopicAssignment().iterator().next().getBatchSizeBytes()); 67 | 68 | } 69 | 70 | } 71 | -------------------------------------------------------------------------------- /memq/src/test/java/com/pinterest/memq/core/clustering/TestMemqGovernor.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.clustering; 17 | 18 | import static org.junit.Assert.assertEquals; 19 | 20 | import java.io.FileNotFoundException; 21 | import java.io.FileReader; 22 | 23 | import org.junit.Test; 24 | 25 | import com.google.gson.Gson; 26 | import com.google.gson.JsonIOException; 27 | import com.google.gson.JsonSyntaxException; 28 | import com.pinterest.memq.commons.protocol.TopicConfig; 29 | 30 | public class TestMemqGovernor { 31 | 32 | @Test 33 | public void testBackwardsCompatibility() throws JsonSyntaxException, JsonIOException, 34 | FileNotFoundException { 35 | Gson gson = new Gson(); 36 | TopicConfig oldConf = gson.fromJson(new FileReader("src/test/resources/old.test_topic.json"), 37 | TopicConfig.class); 38 | TopicConfig newConf = gson.fromJson(new FileReader("src/test/resources/new.test_topic.json"), 39 | TopicConfig.class); 40 | assertEquals("customs3aync2", oldConf.getStorageHandlerName()); 41 | assertEquals("customs3aync2", newConf.getStorageHandlerName()); 42 | } 43 | 44 | } 45 | -------------------------------------------------------------------------------- /memq/src/test/java/com/pinterest/memq/core/integration/TestEnvironmentProvider.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.integration; 17 | 18 | import com.pinterest.memq.core.config.EnvironmentProvider; 19 | 20 | public final class TestEnvironmentProvider extends EnvironmentProvider { 21 | @Override 22 | public String getRack() { 23 | return "local"; 24 | } 25 | 26 | @Override 27 | public String getInstanceType() { 28 | return "2xl"; 29 | } 30 | 31 | @Override 32 | public String getIP() { 33 | return "127.0.0.1"; 34 | } 35 | } -------------------------------------------------------------------------------- /memq/src/test/java/com/pinterest/memq/core/processing/TestOutputHandler.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.processing; 17 | 18 | import java.util.List; 19 | import java.util.Properties; 20 | 21 | import com.codahale.metrics.MetricRegistry; 22 | import com.pinterest.memq.commons.storage.WriteFailedException; 23 | import com.pinterest.memq.commons.storage.StorageHandler; 24 | import com.pinterest.memq.core.commons.Message; 25 | 26 | public class TestOutputHandler implements StorageHandler { 27 | 28 | public TestOutputHandler() { 29 | } 30 | 31 | @Override 32 | public void initWriter(Properties outputHandlerConfig, 33 | String topic, 34 | MetricRegistry registry) throws Exception { 35 | } 36 | 37 | @Override 38 | public void writeOutput(int sizeInBytes, 39 | int checksum, 40 | List messages) throws WriteFailedException { 41 | } 42 | 43 | @Override 44 | public String getReadUrl() { 45 | return null; 46 | } 47 | 48 | } 49 | -------------------------------------------------------------------------------- /memq/src/test/java/com/pinterest/memq/core/rpc/TestAuditMessage.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.rpc; 17 | 18 | public class TestAuditMessage { 19 | 20 | byte[] cluster, hash, topic, hostAddress; 21 | long epoch, id; 22 | int messageCount; 23 | boolean isProducer; 24 | 25 | public TestAuditMessage(byte[] cluster, 26 | byte[] hash, 27 | byte[] topic, 28 | byte[] hostAddress, 29 | long epoch, 30 | long id, 31 | int messageCount, 32 | boolean isProducer) { 33 | super(); 34 | this.cluster = cluster; 35 | this.hash = hash; 36 | this.topic = topic; 37 | this.hostAddress = hostAddress; 38 | this.epoch = epoch; 39 | this.id = id; 40 | this.messageCount = messageCount; 41 | this.isProducer = isProducer; 42 | } 43 | 44 | public byte[] getCluster() { 45 | return cluster; 46 | } 47 | 48 | public void setCluster(byte[] cluster) { 49 | this.cluster = cluster; 50 | } 51 | 52 | public byte[] getHash() { 53 | return hash; 54 | } 55 | 56 | public void setHash(byte[] hash) { 57 | this.hash = hash; 58 | } 59 | 60 | public byte[] getTopic() { 61 | return topic; 62 | } 63 | 64 | public void setTopic(byte[] topic) { 65 | this.topic = topic; 66 | } 67 | 68 | public byte[] getHostAddress() { 69 | return hostAddress; 70 | } 71 | 72 | public void setHostAddress(byte[] hostAddress) { 73 | this.hostAddress = hostAddress; 74 | } 75 | 76 | public long getEpoch() { 77 | return epoch; 78 | } 79 | 80 | public void setEpoch(long epoch) { 81 | this.epoch = epoch; 82 | } 83 | 84 | public long getId() { 85 | return id; 86 | } 87 | 88 | public void setId(long id) { 89 | this.id = id; 90 | } 91 | 92 | public int getMessageCount() { 93 | return messageCount; 94 | } 95 | 96 | } -------------------------------------------------------------------------------- /memq/src/test/java/com/pinterest/memq/core/rpc/TestAuditor.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.rpc; 17 | 18 | import java.io.IOException; 19 | import java.util.ArrayList; 20 | import java.util.List; 21 | import java.util.Properties; 22 | 23 | import com.pinterest.memq.client.commons.audit.Auditor; 24 | 25 | public class TestAuditor extends Auditor { 26 | 27 | private static List auditMessageList = new ArrayList<>(); 28 | 29 | @Override 30 | public void init(Properties props) throws Exception { 31 | } 32 | 33 | @Override 34 | public void auditMessage(byte[] cluster, 35 | byte[] topic, 36 | byte[] hostAddress, 37 | long epoch, 38 | long id, 39 | byte[] hash, 40 | int messageCount, 41 | boolean isProducer, 42 | String clientId) throws IOException { 43 | synchronized(TestAuditor.class) { 44 | auditMessageList.add(new TestAuditMessage(cluster, hash, topic, hostAddress, epoch, id, 45 | messageCount, isProducer)); 46 | } 47 | } 48 | 49 | @Override 50 | public void close() { 51 | } 52 | 53 | public static List getAuditMessageList() { 54 | return auditMessageList; 55 | } 56 | 57 | public static void reset() { 58 | auditMessageList.clear(); 59 | } 60 | 61 | } -------------------------------------------------------------------------------- /memq/src/test/java/com/pinterest/memq/core/utils/TestCoreUtils.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 Pinterest, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * 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 | package com.pinterest.memq.core.utils; 17 | 18 | import static org.junit.Assert.assertEquals; 19 | 20 | import java.util.ArrayList; 21 | import java.util.List; 22 | import java.util.concurrent.Executors; 23 | import java.util.concurrent.ThreadLocalRandom; 24 | import java.util.concurrent.TimeUnit; 25 | 26 | import org.junit.Test; 27 | 28 | import com.google.common.cache.CacheBuilder; 29 | import com.google.common.cache.CacheLoader; 30 | import com.google.common.cache.LoadingCache; 31 | import com.google.common.cache.RemovalListener; 32 | import com.google.common.cache.RemovalNotification; 33 | import com.pinterest.memq.core.commons.Message; 34 | import com.pinterest.memq.core.processing.ProcessingStatus; 35 | 36 | public class TestCoreUtils { 37 | 38 | @Test 39 | public void testBatchSizeCalculator() { 40 | List batch = new ArrayList<>(); 41 | int totalBytes = 0; 42 | for (int i = 0; i < 100; i++) { 43 | int size = ThreadLocalRandom.current().nextInt(800); 44 | totalBytes += size; 45 | byte[] testData = new byte[size]; 46 | Message m = new Message(1024, false); 47 | m.put(testData); 48 | batch.add(m); 49 | } 50 | assertEquals(totalBytes, CoreUtils.batchSizeInBytes(batch)); 51 | } 52 | 53 | @Test 54 | public void testLRUExpiration() throws InterruptedException { 55 | LoadingCache ackMap = CacheBuilder.newBuilder().maximumSize(10000) 56 | .expireAfterWrite(1, TimeUnit.SECONDS) 57 | .removalListener(new RemovalListener() { 58 | 59 | @Override 60 | public void onRemoval(RemovalNotification notification) { 61 | System.out.println("Removed"); 62 | } 63 | }).build(new CacheLoader() { 64 | 65 | @Override 66 | public ProcessingStatus load(Long key) throws Exception { 67 | System.out.println("Loading:" + key); 68 | return ProcessingStatus.FAILED; 69 | } 70 | }); 71 | ackMap.put(1L, ProcessingStatus.PENDING); 72 | assertEquals(1, ackMap.size()); 73 | Executors.newScheduledThreadPool(1, DaemonThreadFactory.INSTANCE) 74 | .scheduleWithFixedDelay(() -> ackMap.cleanUp(), 0, 1, TimeUnit.SECONDS); 75 | Thread.sleep(3000); 76 | assertEquals(0, ackMap.size()); 77 | } 78 | } 79 | -------------------------------------------------------------------------------- /memq/src/test/resources/new.test_topic.json: -------------------------------------------------------------------------------- 1 | { 2 | "topicOrder": 1612218199565, 3 | "bufferSize": 2097152, 4 | "ringBufferSize": 512, 5 | "batchMilliSeconds": 20000, 6 | "batchSizeMB": 15, 7 | "outputParallelism": 60, 8 | "maxDispatchCount": 200, 9 | "topic": "test_topic", 10 | "storageHandlerName": "customs3aync2", 11 | "storageHandlerConfig": { 12 | "delay.max.millis": "10000", 13 | "notificationServerset": "/var/serverset/discovery.testkafka.prod", 14 | "bucket": "abracadabra", 15 | "delay.min.millis": "200", 16 | "disableNotifications": "false", 17 | "notificationBrokerset": "Capacity_B9_P180_0", 18 | "retryTimeoutMillis": "3000", 19 | "notificationTopic": "memq_test_topic", 20 | "retries": "3", 21 | "region": "us-east-1", 22 | "path": "topics/test_topic" 23 | }, 24 | "tickFrequencyMillis": 1000, 25 | "enableBucketing2Processor": true, 26 | "inputTrafficMB": 100.0, 27 | "enableServerHeaderValidation": true 28 | } -------------------------------------------------------------------------------- /memq/src/test/resources/old.test_topic.json: -------------------------------------------------------------------------------- 1 | { 2 | "topicOrder": 1612218199565, 3 | "bufferSize": 2097152, 4 | "ringBufferSize": 512, 5 | "batchMilliSeconds": 20000, 6 | "batchSizeMB": 15, 7 | "outputParallelism": 60, 8 | "maxDispatchCount": 200, 9 | "topic": "test_topic", 10 | "outputHandlerConfig": { 11 | "delay.max.millis": "10000", 12 | "notificationServerset": "/var/serverset/discovery.testkafka.prod", 13 | "bucket": "abracadabra", 14 | "delay.min.millis": "200", 15 | "disableNotifications": "false", 16 | "notificationBrokerset": "Capacity_B9_P180_0", 17 | "retryTimeoutMillis": "3000", 18 | "notificationTopic": "memq_test_topic", 19 | "retries": "3", 20 | "region": "us-east-1", 21 | "path": "topics/test_topic" 22 | }, 23 | "tickFrequencyMillis": 1000, 24 | "enableBucketing2Processor": true, 25 | "outputHandler": "customs3aync2", 26 | "inputTrafficMB": 100.0, 27 | "enableServerHeaderValidation": true 28 | } --------------------------------------------------------------------------------