From 424d17affa327f36a8801b6db175b09acf8f67d7 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Tue, 22 Oct 2024 09:40:54 +0800 Subject: [PATCH] feat: Support new batch model BLEND/PARTITIONED (#1092) * Revert "Add default value for the partitionType to `partition` (#863)" This reverts commit ba24f85c922a61cef14466a913ebd408424dd7cf. * Revert "[Proposal-1] Support Partitioner Refactoring (#845)" This reverts commit a8d9f1d25fbb764b12ce41ba28bc15eb25786859. * feat: Add new batch model (cherry picked from commit 4b94112ef8b84af9b9390dbf02168b834870657c) --- docs/README.md.template | 27 - docs/aws-s3-sink.md | 87 ++- docs/azure-blob-storage-sink.md | 84 ++- docs/cloud-storage-sink.md | 549 ------------------ docs/cloud-storage-sink.png | Bin 19196 -> 0 bytes docs/google-cloud-storage-sink.md | 79 ++- .../io/jcloud/BlobStoreAbstractConfig.java | 62 +- .../io/jcloud/batch/BatchContainer.java | 103 ++++ .../pulsar/io/jcloud/batch/BatchManager.java | 148 +++++ .../BatchModel.java} | 21 +- .../io/jcloud/batch/BlendBatchManager.java | 90 +++ .../jcloud/batch/PartitionedBatchManager.java | 109 ++++ .../legacy => batch}/package-info.java | 2 +- .../pulsar/io/jcloud/format/Format.java | 29 + .../io/jcloud/format/ParquetFormat.java | 2 +- .../{legacy => }/AbstractPartitioner.java | 2 +- .../jcloud/partitioner/LegacyPartitioner.java | 61 -- .../io/jcloud/partitioner/Partitioner.java | 68 ++- .../jcloud/partitioner/PartitionerType.java | 20 +- .../{legacy => }/SimplePartitioner.java | 2 +- .../jcloud/partitioner/TimePartitioner.java | 100 +++- .../jcloud/partitioner/TopicPartitioner.java | 59 -- .../partitioner/legacy/Partitioner.java | 66 --- .../partitioner/legacy/TimePartitioner.java | 112 ---- .../io/jcloud/partitioner/package-info.java | 2 +- .../io/jcloud/sink/BlobStoreAbstractSink.java | 272 ++------- .../sink/CloudStorageGenericRecordSink.java | 5 +- .../jcloud/{ => util}/BytesOutputStream.java | 2 +- ....java => BlobStoreAbstractConfigTest.java} | 41 +- .../{container => }/PulsarContainer.java | 2 +- .../io/jcloud/batch/BatchContainerTest.java | 94 +++ .../io/jcloud/batch/BlendBatchMangerTest.java | 112 ++++ .../batch/PartitionedBatchManagerTest.java | 136 +++++ .../io/jcloud/format/FormatTestBase.java | 2 - .../jcloud/format/JsonFormatMethodTest.java | 1 - .../io/jcloud/format/JsonFormatTest.java | 1 - .../io/jcloud/format/ParquetFormatTest.java | 1 - .../{support => format}/ParquetInputFile.java | 2 +- .../jcloud/{ => format}/PulsarTestBase.java | 4 +- .../io/jcloud/{bo => format}/TestRecord.java | 2 +- .../jcloud/partitioner/PartitionerTest.java | 5 +- ...TypeTest.java => PartitionerTypeTest.java} | 8 +- .../SliceTopicPartitionPartitionerTest.java | 5 +- .../GoogleCloudStorageProviderTest.java | 64 -- ...va => CloudStorageSinkBatchBlendTest.java} | 75 ++- .../CloudStorageSinkBatchPartitionedTest.java | 326 +++++++++++ .../{ => utils}/HexStringUtilsTest.java | 2 +- .../jcloud/{ => utils}/MetadataUtilTest.java | 2 +- 48 files changed, 1599 insertions(+), 1449 deletions(-) delete mode 100644 docs/README.md.template delete mode 100644 docs/cloud-storage-sink.md delete mode 100644 docs/cloud-storage-sink.png create mode 100644 src/main/java/org/apache/pulsar/io/jcloud/batch/BatchContainer.java create mode 100644 src/main/java/org/apache/pulsar/io/jcloud/batch/BatchManager.java rename src/main/java/org/apache/pulsar/io/jcloud/{partitioner/legacy/LegacyPartitionerType.java => batch/BatchModel.java} (51%) create mode 100644 src/main/java/org/apache/pulsar/io/jcloud/batch/BlendBatchManager.java create mode 100644 src/main/java/org/apache/pulsar/io/jcloud/batch/PartitionedBatchManager.java rename src/main/java/org/apache/pulsar/io/jcloud/{partitioner/legacy => batch}/package-info.java (93%) rename src/main/java/org/apache/pulsar/io/jcloud/partitioner/{legacy => }/AbstractPartitioner.java (98%) delete mode 100644 src/main/java/org/apache/pulsar/io/jcloud/partitioner/LegacyPartitioner.java rename src/main/java/org/apache/pulsar/io/jcloud/partitioner/{legacy => }/SimplePartitioner.java (95%) delete mode 100644 src/main/java/org/apache/pulsar/io/jcloud/partitioner/TopicPartitioner.java delete mode 100644 src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/Partitioner.java delete mode 100644 src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/TimePartitioner.java rename src/main/java/org/apache/pulsar/io/jcloud/{ => util}/BytesOutputStream.java (98%) rename src/test/java/org/apache/pulsar/io/jcloud/{ConnectorConfigTest.java => BlobStoreAbstractConfigTest.java} (87%) rename src/test/java/org/apache/pulsar/io/jcloud/{container => }/PulsarContainer.java (98%) create mode 100644 src/test/java/org/apache/pulsar/io/jcloud/batch/BatchContainerTest.java create mode 100644 src/test/java/org/apache/pulsar/io/jcloud/batch/BlendBatchMangerTest.java create mode 100644 src/test/java/org/apache/pulsar/io/jcloud/batch/PartitionedBatchManagerTest.java rename src/test/java/org/apache/pulsar/io/jcloud/{support => format}/ParquetInputFile.java (98%) rename src/test/java/org/apache/pulsar/io/jcloud/{ => format}/PulsarTestBase.java (99%) rename src/test/java/org/apache/pulsar/io/jcloud/{bo => format}/TestRecord.java (96%) rename src/test/java/org/apache/pulsar/io/jcloud/partitioner/{LegacyPartitionerTypeTest.java => PartitionerTypeTest.java} (73%) delete mode 100644 src/test/java/org/apache/pulsar/io/jcloud/provider/GoogleCloudStorageProviderTest.java rename src/test/java/org/apache/pulsar/io/jcloud/sink/{CloudStorageGenericRecordSinkTest.java => CloudStorageSinkBatchBlendTest.java} (82%) create mode 100644 src/test/java/org/apache/pulsar/io/jcloud/sink/CloudStorageSinkBatchPartitionedTest.java rename src/test/java/org/apache/pulsar/io/jcloud/{ => utils}/HexStringUtilsTest.java (97%) rename src/test/java/org/apache/pulsar/io/jcloud/{ => utils}/MetadataUtilTest.java (98%) diff --git a/docs/README.md.template b/docs/README.md.template deleted file mode 100644 index 9a9e9c6f..00000000 --- a/docs/README.md.template +++ /dev/null @@ -1,27 +0,0 @@ -## Pulsar IO :: \ - - - -### Installation - - - -### Configuration - - - -### Examples - - - -### Monitoring - - - -### Troubleshooting - - - -### License - - diff --git a/docs/aws-s3-sink.md b/docs/aws-s3-sink.md index ff67c3f8..89dea0d7 100644 --- a/docs/aws-s3-sink.md +++ b/docs/aws-s3-sink.md @@ -60,7 +60,7 @@ pulsarctl sinks create \ "bucket": "Your bucket name", "region": "Your AWS S3 region", "formatType": "json", - "partitioner": "topic" + "partitionerType": "PARTITION" }' ``` @@ -124,38 +124,38 @@ You can see the object at public/default/{{Your topic name}}-partition-0/xxxx.js Before using the AWS S3 sink connector, you need to configure it. This table outlines the properties and the descriptions. -| Name | Type | Required | Sensitive | Default | Description | -|---------------------------------|---------|----------|-----------|--------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| `provider` | String | True | false | null | The AWS S3 client type, such as `aws-s3`,`s3v2`(`s3v2` uses the AWS client but not the JCloud client). | -| `accessKeyId` | String | True | true | null | The AWS access key ID. It requires permission to write objects. | -| `secretAccessKey` | String | True | true | null | The AWS secret access key. | -| `bucket` | String | True | false | null | The AWS S3 bucket. | -| `formatType` | String | True | false | "json" | The data format type. Available options are `json`, `avro`, `bytes`, or `parquet`. By default, it is set to `json`. | -| `partitioner` | String | False | false | null | The partitioner for partitioning the resulting files. Available options are `topic`, `time` or `legacy`. By default, it's set to `legacy`. Please see [Partitioner](#partitioner) for more details. | -| `partitionerType` | String | False | false | null | The legacy partitioning type. It can be configured by topic partitions or by time. By default, the partition type is configured by topic partitions. It only works when the partitioner is set to `legacy`. | -| `region` | String | False | false | null | The AWS S3 region. Either the endpoint or region must be set. | -| `endpoint` | String | False | false | null | The AWS S3 endpoint. Either the endpoint or region must be set. | -| `role` | String | False | false | null | The AWS role. | -| `roleSessionName` | String | False | false | null | The AWS role session name. | -| `timePartitionPattern` | String | False | false | "yyyy-MM-dd" | The format pattern of the time-based partitioning. For details, refer to the Java date and time format. | -| `timePartitionDuration` | String | False | false | "86400000" | The time interval for time-based partitioning. Support formatted interval string, such as `30d`, `24h`, `30m`, `10s`, and also support number in milliseconds precision, such as `86400000` refers to `24h` or `1d`. | -| `partitionerUseIndexAsOffset` | Boolean | False | false | false | Whether to use the Pulsar's message index as offset or the record sequence. It's recommended if the incoming messages may be batched. The brokers may or not expose the index metadata and, if it's not present on the record, the sequence will be used. See [PIP-70](https://github.com/apache/pulsar/wiki/PIP-70%3A-Introduce-lightweight-broker-entry-metadata) for more details. | -| `batchSize` | int | False | false | 10 | The number of records submitted in batch. | -| `batchTimeMs` | long | False | false | 1000 | The interval for batch submission. | -| `maxBatchBytes` | long | False | false | 10000000 | The maximum number of bytes in a batch. | -| `sliceTopicPartitionPath` | Boolean | False | false | false | When it is set to `true`, split the partitioned topic name into separate folders in the bucket path. | -| `withMetadata` | Boolean | False | false | false | Save message attributes to metadata. | -| `useHumanReadableMessageId` | Boolean | False | false | false | Use a human-readable format string for messageId in message metadata. The messageId is in a format like `ledgerId:entryId:partitionIndex:batchIndex`. Otherwise, the messageId is a Hex-encoded string. | -| `withTopicPartitionNumber` | Boolean | False | false | true | When it is set to `true`, include the topic partition number to the object path. | -| `bytesFormatTypeSeparator` | String | False | false | "0x10" | It is inserted between records for the `formatType` of bytes. By default, it is set to '0x10'. An input record that contains the line separator looks like multiple records in the output object. | -| `pendingQueueSize` | int | False | false | 10 | The number of records buffered in queue. By default, it is equal to `batchSize`. You can set it manually. | -| `useHumanReadableSchemaVersion` | Boolean | False | false | false | Use a human-readable format string for the schema version in the message metadata. If it is set to `true`, the schema version is in plain string format. Otherwise, the schema version is in hex-encoded string format. | -| `skipFailedMessages` | Boolean | False | false | false | Configure whether to skip a message which it fails to be processed. If it is set to `true`, the connector will skip the failed messages by `ack` it. Otherwise, the connector will `fail` the message. | -| `pathPrefix` | String | False | false | false | If it is set, the output files are stored in a folder under the given bucket path. The `pathPrefix` must be in the format of `xx/xxx/`. | -| `avroCodec` | String | False | false | snappy | Compression codec used when formatType=`avro`. Available compression types are: none (no compression), deflate, bzip2, xz, zstandard, snappy. | -| `parquetCodec` | String | False | false | gzip | Compression codec used when formatType=`parquet`. Available compression types are: none (no compression), snappy, gzip, lzo, brotli, lz4, zstd. | -| `jsonAllowNaN` | Boolean | False | false | false | Recognize 'NaN', 'INF', '-INF' as legal floating number values when formatType=`json`. Since JSON specification does not allow such values this is a non-standard feature and disabled by default. | -| `includeTopicToMetadata` | Boolean | False | false | false | Include the topic name to the metadata. | +| Name | Type | Required | Sensitive | Default | Description | +|---------------------------------|---------|----------|-----------|--------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `provider` | String | True | false | null | The AWS S3 client type, such as `aws-s3`,`s3v2`(`s3v2` uses the AWS client but not the JCloud client). | +| `accessKeyId` | String | True | true | null | The AWS access key ID. It requires permission to write objects. | +| `secretAccessKey` | String | True | true | null | The AWS secret access key. | +| `bucket` | String | True | false | null | The AWS S3 bucket. | +| `formatType` | String | True | false | "json" | The data format type. Available options are `json`, `avro`, `bytes`, or `parquet`. By default, it is set to `json`. | +| `partitionerType` | String | False | false | null | The partitioning type. It can be configured by topic `PARTITION` or `TIME`. By default, the partition type is configured by topic partitions. | +| `region` | String | False | false | null | The AWS S3 region. Either the endpoint or region must be set. | +| `endpoint` | String | False | false | null | The AWS S3 endpoint. Either the endpoint or region must be set. | +| `role` | String | False | false | null | The AWS role. | +| `roleSessionName` | String | False | false | null | The AWS role session name. | +| `timePartitionPattern` | String | False | false | "yyyy-MM-dd" | The format pattern of the time-based partitioning. For details, refer to the Java date and time format. | +| `timePartitionDuration` | String | False | false | "86400000" | The time interval for time-based partitioning. Support formatted interval string, such as `30d`, `24h`, `30m`, `10s`, and also support number in milliseconds precision, such as `86400000` refers to `24h` or `1d`. | +| `partitionerUseIndexAsOffset` | Boolean | False | false | false | Whether to use the Pulsar's message index as offset or the record sequence. It's recommended if the incoming messages may be batched. The brokers may or not expose the index metadata and, if it's not present on the record, the sequence will be used. See [PIP-70](https://github.com/apache/pulsar/wiki/PIP-70%3A-Introduce-lightweight-broker-entry-metadata) for more details. | +| `batchSize` | int | False | false | 10 | The number of records submitted in batch. | +| `batchTimeMs` | long | False | false | 1000 | The interval for batch submission. | +| `maxBatchBytes` | long | False | false | 10000000 | The maximum number of bytes in a batch. | +| `batchModel` | Enum | False | false | BLEND | Determines how records are batched. Options: `BLEND`, `PARTITIONED`. The BLEND which combines all topic records into a single batch, optimizing for throughput, and PARTITIONED which batches records separately for each topic, maintaining topic-level separation. Note: When set to PARTITIONED, the connector will cache data up to the size of the number of subscribed topics multiplied by maxBatchBytes. This means you need to anticipate the connector's memory requirements in advance. | +| `pendingQueueSize` | int | False | false | 10 | The number of records buffered in queue. By default, it is equal to `batchSize`. You can set it manually. | +| `withMetadata` | Boolean | False | false | false | Save message attributes to metadata. | +| `includeTopicToMetadata` | Boolean | False | false | false | Include the topic name to the metadata. | +| `sliceTopicPartitionPath` | Boolean | False | false | false | When it is set to `true`, split the partitioned topic name into separate folders in the bucket path. | +| `useHumanReadableMessageId` | Boolean | False | false | false | Use a human-readable format string for messageId in message metadata. The messageId is in a format like `ledgerId:entryId:partitionIndex:batchIndex`. Otherwise, the messageId is a Hex-encoded string. | +| `withTopicPartitionNumber` | Boolean | False | false | true | When it is set to `true`, include the topic partition number to the object path. | +| `bytesFormatTypeSeparator` | String | False | false | "0x10" | It is inserted between records for the `formatType` of bytes. By default, it is set to '0x10'. An input record that contains the line separator looks like multiple records in the output object. | +| `useHumanReadableSchemaVersion` | Boolean | False | false | false | Use a human-readable format string for the schema version in the message metadata. If it is set to `true`, the schema version is in plain string format. Otherwise, the schema version is in hex-encoded string format. | +| `skipFailedMessages` | Boolean | False | false | false | Configure whether to skip a message which it fails to be processed. If it is set to `true`, the connector will skip the failed messages by `ack` it. Otherwise, the connector will `fail` the message. | +| `pathPrefix` | String | False | false | false | If it is set, the output files are stored in a folder under the given bucket path. The `pathPrefix` must be in the format of `xx/xxx/`. | +| `avroCodec` | String | False | false | snappy | Compression codec used when formatType=`avro`. Available compression types are: none (no compression), deflate, bzip2, xz, zstandard, snappy. | +| `parquetCodec` | String | False | false | gzip | Compression codec used when formatType=`parquet`. Available compression types are: none (no compression), snappy, gzip, lzo, brotli, lz4, zstd. | +| `jsonAllowNaN` | Boolean | False | false | false | Recognize 'NaN', 'INF', '-INF' as legal floating number values when formatType=`json`. Since JSON specification does not allow such values this is a non-standard feature and disabled by default. | ## Advanced features @@ -229,29 +229,16 @@ The reason for this is that the sink will only acknowledge messages after they a If this limit is lower or close to `batchSize`, the sink never receives enough messages to trigger a flush based on the amount of messages. In this case please ensure the `maxUnackedMessagesPerConsumer` set in the broker configuration is sufficiently larger than the `batchSize` setting of the sink. -### Partitioner +### Partitioner Type -The partitioner is used for partitioning the data into different files in the cloud storage. -There are three types of partitioner: +There are two types of partitioner: -- **Topic Partitioner**: Messages are partitioned according to the pre-existing partitions in the Pulsar topics. For - instance, a message for the topic `public/default/my-topic-partition-0` would be directed to the - file `public/default/my-topic-partition-0/xxx.json`, where `xxx` signifies the earliest message offset in this file. -- **Time Partitioner**: Messages are partitioned based on the timestamp at the time of flushing. For the aforementioned - message, it would be directed to the file `1703037311.json`, where `1703037311` represents the flush timestamp of the - first message in this file. -- **Legacy Partitioner**: This type reverts to the old partitioner behavior. The legacy configuration `partitionerType` would be respected. - -#### Legacy Partitioner - -There are two types of legacy partitioner: - -- **Simple partitioner**: This is the default partitioning method based on Pulsar partitions. In other words, data is +- **PARTITION**: This is the default partitioning method based on Pulsar partitions. In other words, data is partitioned according to the pre-existing partitions in Pulsar topics. For instance, a message for the topic `public/default/my-topic-partition-0` would be directed to the file `public/default/my-topic-partition-0/xxx.json`, where `xxx` signifies the earliest message offset in this file. -- **Time partitioner**: Data is partitioned according to the time it was flushed. Using the previous message as an +- **TIME**: Data is partitioned according to the time it was flushed. Using the previous message as an example, if it was received on 2023-12-20, it would be directed to `public/default/my-topic-partition-0/2023-12-20/xxx.json`, where `xxx` also denotes the earliest message offset in this file. diff --git a/docs/azure-blob-storage-sink.md b/docs/azure-blob-storage-sink.md index 4b5e1801..704dddf7 100644 --- a/docs/azure-blob-storage-sink.md +++ b/docs/azure-blob-storage-sink.md @@ -42,7 +42,7 @@ pulsarctl sinks create \ "provider": "azure-blob-storage", "bucket": "Your container name", "formatType": "json", - "partitioner": "topic" + "partitionerType": "PARTITION" }' ``` @@ -106,36 +106,37 @@ You can see the object at public/default/{{Your topic name}}-partition-0/xxxx.js Before using the Azure Blob Storage sink connector, you need to configure it. This table outlines the properties and the descriptions. -| Name | Type | Required | Sensitive | Default | Description | -|---------------------------------|---------|----------|-----------|--------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| `provider` | String | True | false | null | The Cloud Storage type, Azure Blob Storage only supports the `azure-blob-storage` provider. | -| `bucket` | String | True | false | null | The Azure Blob Storage container name. | -| `formatType` | String | True | false | "json" | The data format type. Available options are `json`, `avro`, `bytes`, or `parquet`. By default, it is set to `json`. | -| `partitioner` | String | False | false | null | The partitioner for partitioning the resulting files. Available options are `topic`, `time` or `legacy`. By default, it's set to `legacy`. Please see [Partitioner](#partitioner) for more details. | -| `partitionerType` | String | False | false | null | The legacy partitioning type. It can be configured by topic partitions or by time. By default, the partition type is configured by topic partitions. It only works when the partitioner is set to `legacy`. || `azureStorageAccountConnectionString` | String | False | true | "" | The Azure Blob Storage connection string. Required when authenticating via connection string. | -| `azureStorageAccountSASToken` | String | False | true | "" | The Azure Blob Storage account SAS token. Required when authenticating via SAS token. | -| `azureStorageAccountName` | String | False | true | "" | The Azure Blob Storage account name. Required when authenticating via account name and account key. | -| `azureStorageAccountKey` | String | False | true | "" | The Azure Blob Storage account key. Required when authenticating via account name and account key. | -| `endpoint` | String | False | false | null | The Azure Blob Storage endpoint. Required when authenticating via account name or SAS token. | -| `timePartitionPattern` | String | False | false | "yyyy-MM-dd" | The format pattern of the time-based partitioning. For details, refer to the Java date and time format. | -| `timePartitionDuration` | String | False | false | "86400000" | The time interval for time-based partitioning. Support formatted interval string, such as `30d`, `24h`, `30m`, `10s`, and also support number in milliseconds precision, such as `86400000` refers to `24h` or `1d`. | -| `partitionerUseIndexAsOffset` | Boolean | False | false | false | Whether to use the Pulsar's message index as offset or the record sequence. It's recommended if the incoming messages may be batched. The brokers may or not expose the index metadata and, if it's not present on the record, the sequence will be used. See [PIP-70](https://github.com/apache/pulsar/wiki/PIP-70%3A-Introduce-lightweight-broker-entry-metadata) for more details. | -| `batchSize` | int | False | false | 10 | The number of records submitted in batch. | -| `batchTimeMs` | long | False | false | 1000 | The interval for batch submission. | -| `maxBatchBytes` | long | False | false | 10000000 | The maximum number of bytes in a batch. | -| `sliceTopicPartitionPath` | Boolean | False | false | false | When it is set to `true`, split the partitioned topic name into separate folders in the bucket path. | -| `withMetadata` | Boolean | False | false | false | Save message attributes to metadata. | -| `useHumanReadableMessageId` | Boolean | False | false | false | Use a human-readable format string for messageId in message metadata. The messageId is in a format like `ledgerId:entryId:partitionIndex:batchIndex`. Otherwise, the messageId is a Hex-encoded string. | -| `withTopicPartitionNumber` | Boolean | False | false | true | When it is set to `true`, include the topic partition number to the object path. | -| `bytesFormatTypeSeparator` | String | False | false | "0x10" | It is inserted between records for the `formatType` of bytes. By default, it is set to '0x10'. An input record that contains the line separator looks like multiple records in the output object. | -| `pendingQueueSize` | int | False | false | 10 | The number of records buffered in queue. By default, it is equal to `batchSize`. You can set it manually. | -| `useHumanReadableSchemaVersion` | Boolean | False | false | false | Use a human-readable format string for the schema version in the message metadata. If it is set to `true`, the schema version is in plain string format. Otherwise, the schema version is in hex-encoded string format. | -| `skipFailedMessages` | Boolean | False | false | false | Configure whether to skip a message which it fails to be processed. If it is set to `true`, the connector will skip the failed messages by `ack` it. Otherwise, the connector will `fail` the message. | -| `pathPrefix` | String | False | false | false | If it is set, the output files are stored in a folder under the given bucket path. The `pathPrefix` must be in the format of `xx/xxx/`. | -| `avroCodec` | String | False | false | snappy | Compression codec used when formatType=`avro`. Available compression types are: none (no compression), deflate, bzip2, xz, zstandard, snappy. | -| `parquetCodec` | String | False | false | gzip | Compression codec used when formatType=`parquet`. Available compression types are: none (no compression), snappy, gzip, lzo, brotli, lz4, zstd. | -| `jsonAllowNaN` | Boolean | False | false | false | Recognize 'NaN', 'INF', '-INF' as legal floating number values when formatType=`json`. Since JSON specification does not allow such values this is a non-standard feature and disabled by default. | -| `includeTopicToMetadata` | Boolean | False | false | false | Include the topic name to the metadata. | +| Name | Type | Required | Sensitive | Default | Description | +|---------------------------------------|---------|----------|-----------|--------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `provider` | String | True | false | null | The Cloud Storage type, Azure Blob Storage only supports the `azure-blob-storage` provider. | +| `bucket` | String | True | false | null | The Azure Blob Storage container name. | +| `formatType` | String | True | false | "json" | The data format type. Available options are `json`, `avro`, `bytes`, or `parquet`. By default, it is set to `json`. | +| `partitionerType` | String | False | false | null | The partitioning type. It can be configured by topic `PARTITION` or `TIME`. By default, the partition type is configured by topic partitions. | +| `azureStorageAccountConnectionString` | String | False | true | "" | The Azure Blob Storage connection string. Required when authenticating via connection string. | +| `azureStorageAccountSASToken` | String | False | true | "" | The Azure Blob Storage account SAS token. Required when authenticating via SAS token. | +| `azureStorageAccountName` | String | False | true | "" | The Azure Blob Storage account name. Required when authenticating via account name and account key. | +| `azureStorageAccountKey` | String | False | true | "" | The Azure Blob Storage account key. Required when authenticating via account name and account key. | +| `endpoint` | String | False | false | null | The Azure Blob Storage endpoint. Required when authenticating via account name or SAS token. | +| `timePartitionPattern` | String | False | false | "yyyy-MM-dd" | The format pattern of the time-based partitioning. For details, refer to the Java date and time format. | +| `timePartitionDuration` | String | False | false | "86400000" | The time interval for time-based partitioning. Support formatted interval string, such as `30d`, `24h`, `30m`, `10s`, and also support number in milliseconds precision, such as `86400000` refers to `24h` or `1d`. | +| `partitionerUseIndexAsOffset` | Boolean | False | false | false | Whether to use the Pulsar's message index as offset or the record sequence. It's recommended if the incoming messages may be batched. The brokers may or not expose the index metadata and, if it's not present on the record, the sequence will be used. See [PIP-70](https://github.com/apache/pulsar/wiki/PIP-70%3A-Introduce-lightweight-broker-entry-metadata) for more details. | +| `batchSize` | int | False | false | 10 | The number of records submitted in batch. | +| `batchTimeMs` | long | False | false | 1000 | The interval for batch submission. | +| `maxBatchBytes` | long | False | false | 10000000 | The maximum number of bytes in a batch. | +| `batchModel` | Enum | False | false | BLEND | Determines how records are batched. Options: `BLEND`, `PARTITIONED`. The BLEND which combines all topic records into a single batch, optimizing for throughput, and PARTITIONED which batches records separately for each topic, maintaining topic-level separation. Note: When set to PARTITIONED, the connector will cache data up to the size of the number of subscribed topics multiplied by maxBatchBytes. This means you need to anticipate the connector's memory requirements in advance. | +| `pendingQueueSize` | int | False | false | 10 | The number of records buffered in queue. By default, it is equal to `batchSize`. You can set it manually. | +| `withMetadata` | Boolean | False | false | false | Save message attributes to metadata. | +| `includeTopicToMetadata` | Boolean | False | false | false | Include the topic name to the metadata. | +| `sliceTopicPartitionPath` | Boolean | False | false | false | When it is set to `true`, split the partitioned topic name into separate folders in the bucket path. | +| `useHumanReadableMessageId` | Boolean | False | false | false | Use a human-readable format string for messageId in message metadata. The messageId is in a format like `ledgerId:entryId:partitionIndex:batchIndex`. Otherwise, the messageId is a Hex-encoded string. | +| `withTopicPartitionNumber` | Boolean | False | false | true | When it is set to `true`, include the topic partition number to the object path. | +| `bytesFormatTypeSeparator` | String | False | false | "0x10" | It is inserted between records for the `formatType` of bytes. By default, it is set to '0x10'. An input record that contains the line separator looks like multiple records in the output object. | +| `useHumanReadableSchemaVersion` | Boolean | False | false | false | Use a human-readable format string for the schema version in the message metadata. If it is set to `true`, the schema version is in plain string format. Otherwise, the schema version is in hex-encoded string format. | +| `skipFailedMessages` | Boolean | False | false | false | Configure whether to skip a message which it fails to be processed. If it is set to `true`, the connector will skip the failed messages by `ack` it. Otherwise, the connector will `fail` the message. | +| `pathPrefix` | String | False | false | false | If it is set, the output files are stored in a folder under the given bucket path. The `pathPrefix` must be in the format of `xx/xxx/`. | +| `avroCodec` | String | False | false | snappy | Compression codec used when formatType=`avro`. Available compression types are: none (no compression), deflate, bzip2, xz, zstandard, snappy. | +| `parquetCodec` | String | False | false | gzip | Compression codec used when formatType=`parquet`. Available compression types are: none (no compression), snappy, gzip, lzo, brotli, lz4, zstd. | +| `jsonAllowNaN` | Boolean | False | false | false | Recognize 'NaN', 'INF', '-INF' as legal floating number values when formatType=`json`. Since JSON specification does not allow such values this is a non-standard feature and disabled by default. | There are three methods to authenticate with Azure Blob Storage: 1. `azureStorageAccountConnectionString`: This method involves using the Azure Blob Storage connection string for authentication. It's the simplest method as it only requires the connection string. @@ -215,29 +216,16 @@ The reason for this is that the sink will only acknowledge messages after they a If this limit is lower or close to `batchSize`, the sink never receives enough messages to trigger a flush based on the amount of messages. In this case please ensure the `maxUnackedMessagesPerConsumer` set in the broker configuration is sufficiently larger than the `batchSize` setting of the sink. -### Partitioner +### Partitioner Type -The partitioner is used for partitioning the data into different files in the cloud storage. -There are three types of partitioner: +There are two types of partitioner: -- **Topic Partitioner**: Messages are partitioned according to the pre-existing partitions in the Pulsar topics. For - instance, a message for the topic `public/default/my-topic-partition-0` would be directed to the - file `public/default/my-topic-partition-0/xxx.json`, where `xxx` signifies the earliest message offset in this file. -- **Time Partitioner**: Messages are partitioned based on the timestamp at the time of flushing. For the aforementioned - message, it would be directed to the file `1703037311.json`, where `1703037311` represents the flush timestamp of the - first message in this file. -- **Legacy Partitioner**: This type reverts to the old partitioner behavior. The legacy configuration `partitionerType` would be respected. - -#### Legacy Partitioner - -There are two types of legacy partitioner: - -- **Simple partitioner**: This is the default partitioning method based on Pulsar partitions. In other words, data is +- **PARTITION**: This is the default partitioning method based on Pulsar partitions. In other words, data is partitioned according to the pre-existing partitions in Pulsar topics. For instance, a message for the topic `public/default/my-topic-partition-0` would be directed to the file `public/default/my-topic-partition-0/xxx.json`, where `xxx` signifies the earliest message offset in this file. -- **Time partitioner**: Data is partitioned according to the time it was flushed. Using the previous message as an +- **TIME**: Data is partitioned according to the time it was flushed. Using the previous message as an example, if it was received on 2023-12-20, it would be directed to `public/default/my-topic-partition-0/2023-12-20/xxx.json`, where `xxx` also denotes the earliest message offset in this file. \ No newline at end of file diff --git a/docs/cloud-storage-sink.md b/docs/cloud-storage-sink.md deleted file mode 100644 index d960afcd..00000000 --- a/docs/cloud-storage-sink.md +++ /dev/null @@ -1,549 +0,0 @@ ---- -dockerfile: "https://hub.docker.com/r/streamnative/pulsar-io-cloud-storage" -alias: Cloud Storage Sink Connector ---- - -The Cloud Storage sink connector supports exporting data from Pulsar topics to cloud storage (such as AWS S3 and Google GCS) either in Avro, JSON, Parquet or other formats. According to your environment, the Cloud Storage sink connector can guarantee exactly-once support for exporting data to cloud storage. - -![](/docs/cloud-storage-sink.png) - -# How to get - -You can get the Cloud Storage sink connector using one of the following methods: - -## Use it with Function Worker - -- Download the NAR package from [here](https://github.com/streamnative/pulsar-io-cloud-storage/releases/download/v{{connector:version}}/pulsar-io-cloud-storage-{{connector:version}}.nar). - -- Build it from the source code. - - 1. Clone the source code to your machine. - - ```bash - git clone https://github.com/streamnative/pulsar-io-cloud-storage.git - ``` - - 2. Assume that `PULSAR_IO_CLOUD_STORAGE_HOME` is the home directory for the `pulsar-io-cloud-storage` repo. Build the connector in the `${PULSAR_IO_CLOUD_STORAGE_HOME}` directory. - - ```bash - mvn clean install -DskipTests - ``` - - After the connector is successfully built, a `NAR` package is generated under the `target` directory. - - ```bash - ls target - pulsar-io-cloud-storage-{{connector:version}}.nar - ``` - -## Use it with Function Mesh - -Pull the Cloud Storage connector Docker image from [here](https://hub.docker.com/r/streamnative/pulsar-io-cloud-storage). - -# How to configure - -Before using the Cloud Storage sink connector, you need to configure it. - -You can create a configuration file (JSON or YAML) to set the following properties. - -### Storage provider: AWS S3 - -| Name | Type | Required | Default | Description | -|---------------------------------|---------|----------|--------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| `provider` | String | True | null | The Cloud Storage type, such as `aws-s3`,`s3v2`(`s3v2` uses the AWS client but not the JCloud client). | -| `accessKeyId` | String | True | null | The Cloud Storage access key ID. It requires permission to write objects. | -| `secretAccessKey` | String | True | null | The Cloud Storage secret access key. | -| `role` | String | False | null | The Cloud Storage role. | -| `roleSessionName` | String | False | null | The Cloud Storage role session name. | -| `endpoint` | String | True | null | The Cloud Storage endpoint. | -| `bucket` | String | True | null | The Cloud Storage bucket. | -| `formatType` | String | True | "json" | The data format type. Available options are `json`, `avro`, `bytes`, or `parquet`. By default, it is set to `json`. | -| `partitionerType` | String | False | "partition" | The partitioning type. It can be configured by topic partitions or by time. By default, the partition type is configured by topic partitions. | -| `timePartitionPattern` | String | False | "yyyy-MM-dd" | The format pattern of the time-based partitioning. For details, refer to the Java date and time format. | -| `timePartitionDuration` | String | False | "86400000" | The time interval for time-based partitioning. Support formatted interval string, such as `30d`, `24h`, `30m`, `10s`, and also support number in milliseconds precision, such as `86400000` refers to `24h` or `1d`. | -| `partitionerUseIndexAsOffset` | Boolean | False | false | Whether to use the Pulsar's message index as offset or the record sequence. It's recommended if the incoming messages may be batched. The brokers may or not expose the index metadata and, if it's not present on the record, the sequence will be used. See [PIP-70](https://github.com/apache/pulsar/wiki/PIP-70%3A-Introduce-lightweight-broker-entry-metadata) for more details. | -| `batchSize` | int | False | 10 | The number of records submitted in batch. | -| `batchTimeMs` | long | False | 1000 | The interval for batch submission. | -| `maxBatchBytes` | long | False | 10000000 | The maximum number of bytes in a batch. | -| `sliceTopicPartitionPath` | Boolean | False | false | When it is set to `true`, split the partitioned topic name into separate folders in the bucket path. | -| `withMetadata` | Boolean | False | false | Save message attributes to metadata. | -| `useHumanReadableMessageId` | Boolean | False | false | Use a human-readable format string for messageId in message metadata. The messageId is in a format like `ledgerId:entryId:partitionIndex:batchIndex`. Otherwise, the messageId is a Hex-encoded string. | -| `withTopicPartitionNumber` | Boolean | False | true | When it is set to `true`, include the topic partition number to the object path. | -| `bytesFormatTypeSeparator` | String | False | "0x10" | It is inserted between records for the `formatType` of bytes. By default, it is set to '0x10'. An input record that contains the line separator looks like multiple records in the output object. | -| `pendingQueueSize` | int | False | 10 | The number of records buffered in queue. By default, it is equal to `batchSize`. You can set it manually. | -| `useHumanReadableSchemaVersion` | Boolean | False | false | Use a human-readable format string for the schema version in the message metadata. If it is set to `true`, the schema version is in plain string format. Otherwise, the schema version is in hex-encoded string format. | -| `skipFailedMessages` | Boolean | False | false | Configure whether to skip a message which it fails to be processed. If it is set to `true`, the connector will skip the failed messages by `ack` it. Otherwise, the connector will `fail` the message. | -| `pathPrefix` | String | False | false | If it is set, the output files are stored in a folder under the given bucket path. The `pathPrefix` must be in the format of `xx/xxx/`. | -| `avroCodec` | String | False | snappy | Compression codec used when formatType=`avro`. Available compression types are: null (no compression), deflate, bzip2, xz, zstandard, snappy. | -| `parquetCodec` | String | False | gzip | Compression codec used when formatType=`parquet`. Available compression types are: null (no compression), snappy, gzip, lzo, brotli, lz4, zstd. | -| `jsonAllowNaN` | Boolean | False | false | Recognize 'NaN', 'INF', '-INF' as legal floating number values when formatType=`json`. Since JSON specification does not allow such values this is a non-standard feature and disabled by default. | -| `includeTopicToMetadata` | Boolean | False | false | Include the topic name to the metadata. | - -The provided AWS credentials must have permissions to access AWS resources. -To use the Cloud Storage sink connector, the suggested permission policies for AWS S3 are: -- `s3:AbortMultipartUpload` -- `s3:GetObject*` -- `s3:PutObject*` -- `s3:List*` - -If you do not want to provide `region` in the configuration, you should enable the `s3:GetBucketLocation` permission policy as well. - -### Storage provider: Google Cloud Storage - -| Name | Type | Required | Default | Description | -|-----------------------------------|---------|----------|--------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| `provider` | String | True | null | The Cloud Storage type, google cloud storage only supports the `google-cloud-storage` provider. | -| `gcsServiceAccountKeyFilePath` | String | False | "" | Path to the GCS credentials file. If empty, the credentials file will be read from the `GOOGLE_APPLICATION_CREDENTIALS` environment variable. | -| `gcsServiceAccountKeyFileContent` | String | False | "" | The contents of the JSON service key file. If empty, credentials are read from `gcsServiceAccountKeyFilePath` file. | -| `bucket` | String | True | null | The Cloud Storage bucket. | -| `formatType` | String | True | "json" | The data format type. Available options are `json`, `avro`, `bytes`, or `parquet`. By default, it is set to `json`. | -| `partitionerType` | String | False | "partition" | The partitioning type. It can be configured by topic partitions or by time. By default, the partition type is configured by topic partitions. | -| `timePartitionPattern` | String | False | "yyyy-MM-dd" | The format pattern of the time-based partitioning. For details, refer to the Java date and time format. | -| `timePartitionDuration` | String | False | "86400000" | The time interval for time-based partitioning. Support formatted interval string, such as `30d`, `24h`, `30m`, `10s`, and also support number in milliseconds precision, such as `86400000` refers to `24h` or `1d`. | -| `partitionerUseIndexAsOffset` | Boolean | False | false | Whether to use the Pulsar's message index as offset or the record sequence. It's recommended if the incoming messages may be batched. The brokers may or not expose the index metadata and, if it's not present on the record, the sequence will be used. See [PIP-70](https://github.com/apache/pulsar/wiki/PIP-70%3A-Introduce-lightweight-broker-entry-metadata) for more details. | -| `batchSize` | int | False | 10 | The number of records submitted in batch. | -| `batchTimeMs` | long | False | 1000 | The interval for batch submission. | -| `maxBatchBytes` | long | False | 10000000 | The maximum number of bytes in a batch. | -| `sliceTopicPartitionPath` | Boolean | False | false | When it is set to `true`, split the partitioned topic name into separate folders in the bucket path. | -| `withMetadata` | Boolean | False | false | Save message attributes to metadata. | -| `useHumanReadableMessageId` | Boolean | False | false | Use a human-readable format string for messageId in message metadata. The messageId is in a format like `ledgerId:entryId:partitionIndex:batchIndex`. Otherwise, the messageId is a Hex-encoded string. | -| `withTopicPartitionNumber` | Boolean | False | true | When it is set to `true`, include the topic partition number to the object path. | -| `bytesFormatTypeSeparator` | String | False | "0x10" | It is inserted between records for the `formatType` of bytes. By default, it is set to '0x10'. An input record that contains the line separator looks like multiple records in the output object. | -| `pendingQueueSize` | int | False | 10 | The number of records buffered in queue. By default, it is equal to `batchSize`. You can set it manually. | -| `useHumanReadableSchemaVersion` | Boolean | False | false | Use a human-readable format string for the schema version in the message metadata. If it is set to `true`, the schema version is in plain string format. Otherwise, the schema version is in hex-encoded string format. | -| `skipFailedMessages` | Boolean | False | false | Configure whether to skip a message which it fails to be processed. If it is set to `true`, the connector will skip the failed messages by `ack` it. Otherwise, the connector will `fail` the message. | -| `pathPrefix` | String | False | false | If it is set, the output files are stored in a folder under the given bucket path. The `pathPrefix` must be in the format of `xx/xxx/`. | -| `avroCodec` | String | False | snappy | Compression codec used when formatType=`avro`. Available compression types are: null (no compression), deflate, bzip2, xz, zstandard, snappy. | -| `parquetCodec` | String | False | gzip | Compression codec used when formatType=`parquet`. Available compression types are: null (no compression), snappy, gzip, lzo, brotli, lz4, zstd. | -| `jsonAllowNaN` | Boolean | False | false | Recognize 'NaN', 'INF', '-INF' as legal floating number values when formatType=`json`. Since JSON specification does not allow such values this is a non-standard feature and disabled by default. | - -#### Storage Provider: Azure Blob Storage - -| Name | Type | Required | Default | Description | -|---------------------------------------|---------|----------|--------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| `provider` | String | True | null | The Cloud Storage type. Azure Blob Storage only supports the `azure-blob-storage` provider. | -| `azureStorageAccountSASToken` | String | True | "" | The Azure Blob Storage account SAS token. Required when authenticating via SAS token. | -| `azureStorageAccountName` | String | True | "" | The Azure Blob Storage account name. Required when authenticating via account name and account key. | -| `azureStorageAccountKey` | String | True | "" | The Azure Blob Storage account key. Required when authenticating via account name and account key. | -| `azureStorageAccountConnectionString` | String | True | "" | The Azure Blob Storage connection string. Required when authenticating via connection string. | -| `endpoint` | String | True | null | The Azure Blob Storage endpoint. | -| `bucket` | String | True | null | The Cloud Storage bucket. | -| `formatType` | String | True | "json" | The data format type. Available options are `json`, `avro`, `bytes`, or `parquet`. By default, it is set to `json`. | -| `partitionerType` | String | False | "partition" | The partitioning type. It can be configured by topic partitions or by time. By default, the partition type is configured by topic partitions. | -| `timePartitionPattern` | String | False | "yyyy-MM-dd" | The format pattern of the time-based partitioning. For details, refer to the Java date and time format. | -| `timePartitionDuration` | String | False | "86400000" | The time interval for time-based partitioning. Support formatted interval string, such as `30d`, `24h`, `30m`, `10s`, and also support number in milliseconds precision, such as `86400000` refers to `24h` or `1d`. | -| `partitionerUseIndexAsOffset` | Boolean | False | false | Whether to use the Pulsar's message index as offset or the record sequence. It's recommended if the incoming messages may be batched. The brokers may or not expose the index metadata and, if it's not present on the record, the sequence will be used. See [PIP-70](https://github.com/apache/pulsar/wiki/PIP-70%3A-Introduce-lightweight-broker-entry-metadata) for more details. | -| `batchSize` | int | False | 10 | The number of records submitted in batch. | -| `batchTimeMs` | long | False | 1000 | The interval for batch submission. | -| `maxBatchBytes` | long | False | 10000000 | The maximum number of bytes in a batch. | -| `sliceTopicPartitionPath` | Boolean | False | false | When it is set to `true`, split the partitioned topic name into separate folders in the bucket path. | -| `withMetadata` | Boolean | False | false | Save message attributes to metadata. | -| `useHumanReadableMessageId` | Boolean | False | false | Use a human-readable format string for messageId in message metadata. The messageId is in a format like `ledgerId:entryId:partitionIndex:batchIndex`. Otherwise, the messageId is a Hex-encoded string. | -| `withTopicPartitionNumber` | Boolean | False | true | When it is set to `true`, include the topic partition number to the object path. | -| `bytesFormatTypeSeparator` | String | False | "0x10" | It is inserted between records for the `formatType` of bytes. By default, it is set to '0x10'. An input record that contains the line separator looks like multiple records in the output object. | -| `pendingQueueSize` | int | False | 10 | The number of records buffered in queue. By default, it is equal to `batchSize`. You can set it manually. | -| `useHumanReadableSchemaVersion` | Boolean | False | false | Use a human-readable format string for the schema version in the message metadata. If it is set to `true`, the schema version is in plain string format. Otherwise, the schema version is in hex-encoded string format. | -| `skipFailedMessages` | Boolean | False | false | Configure whether to skip a message which it fails to be processed. If it is set to `true`, the connector will skip the failed messages by `ack` it. Otherwise, the connector will `fail` the message. | -| `pathPrefix` | String | False | false | If it is set, the output files are stored in a folder under the given bucket path. The `pathPrefix` must be in the format of `xx/xxx/`. | -| `avroCodec` | String | False | snappy | Compression codec used when formatType=`avro`. Available compression types are: null (no compression), deflate, bzip2, xz, zstandard, snappy. | -| `parquetCodec` | String | False | gzip | Compression codec used when formatType=`parquet`. Available compression types are: null (no compression), snappy, gzip, lzo, brotli, lz4, zstd. | -| `jsonAllowNaN` | Boolean | False | false | Recognize 'NaN', 'INF', '-INF' as legal floating number values when formatType=`json`. Since JSON specification does not allow such values this is a non-standard feature and disabled by default. | - -### Data format types - -Cloud Storage Sink Connector provides multiple output format options, including JSON, Avro, Bytes, or Parquet. The default format is JSON. -With current implementation, there are some limitations for different formats: - -This table lists the Pulsar Schema types supported by the writers. - -| Pulsar Schema | Writer: Avro | Writer: JSON | Writer: Parquet | Writer: Bytes | -|----------------|--------------|--------------|-----------------|---------------| -| Primitive | ✗ | ✔ * | ✗ | ✔ | -| Avro | ✔ | ✔ | ✔ | ✔ | -| Json | ✔ | ✔ | ✔ | ✔ | -| Protobuf ** | ✔ | ✔ | ✔ | ✔ | -| ProtobufNative | ✔ *** | ✗ | ✔ | ✔ | -> *: The JSON writer will try to convert the data with a `String` or `Bytes` schema to JSON-format data if convertable. -> -> **: The Protobuf schema is based on the Avro schema. It uses Avro as an intermediate format, so it may not provide the best effort conversion. -> -> ***: The ProtobufNative record holds the Protobuf descriptor and the message. When writing to Avro format, the connector uses [avro-protobuf](https://github.com/apache/avro/tree/master/lang/java/protobuf) to do the conversion. - -This table lists the support of `withMetadata` configurations for different writer formats: - -| Writer Format | `withMetadata` | -|---------------|----------------| -| Avro | ✔ | -| JSON | ✔ | -| Parquet | ✔ * | -| Bytes | ✗ | - -> *: When using `Parquet` with `PROTOBUF_NATIVE` format, the connector will write the messages with `DynamicMessage` format. When `withMetadata` is set to `true`, the connector will add `__message_metadata__` to the messages with `PulsarIOCSCProtobufMessageMetadata` format. -> -> For example, if a message `User` has the following schema: -> ```protobuf -> syntax = "proto3"; -> message User { -> string name = 1; -> int32 age = 2; -> } -> ``` -> -> When `withMetadata` is set to `true`, the connector will write the message `DynamicMessage` with the following schema: -> ```protobuf -> syntax = "proto3"; -> message PulsarIOCSCProtobufMessageMetadata { -> map properties = 1; -> string schema_version = 2; -> string message_id = 3; -> } -> message User { -> string name = 1; -> int32 age = 2; -> PulsarIOCSCProtobufMessageMetadata __message_metadata__ = 3; -> } -> ``` -> - -## Configure it with Function Worker - -You can create a configuration file (JSON or YAML) to set the properties as below. - -**Example** - -* JSON - - ```json - { - "tenant": "public", - "namespace": "default", - "name": "cloud-storage-sink", - "inputs": [ - "user-avro-topic" - ], - "archive": "connectors/pulsar-io-cloud-storage-0.0.1.nar", - "parallelism": 1, - "configs": { - "provider": "aws-s3", - "accessKeyId": "accessKeyId", - "secretAccessKey": "secretAccessKey", - "role": "none", - "roleSessionName": "none", - "bucket": "testBucket", - "region": "local", - "endpoint": "us-standard", - "formatType": "parquet", - "partitionerType": "time", - "timePartitionPattern": "yyyy-MM-dd", - "timePartitionDuration": "1d", - "batchSize": 10, - "batchTimeMs": 1000 - } - } - ``` - -* YAML - - ```yaml - tenant: "public" - namespace: "default" - name: "Cloud Storage-sink" - inputs: - - "user-avro-topic" - archive: "connectors/pulsar-io-cloud-storage-0.0.1.nar" - parallelism: 1 - - configs: - provider: "aws-s3", - accessKeyId: "accessKeyId" - secretAccessKey: "secretAccessKey" - role: "none" - roleSessionName: "none" - bucket: "testBucket" - region: "local" - endpoint: "us-standard" - formatType: "parquet" - partitionerType: "time" - timePartitionPattern: "yyyy-MM-dd" - timePartitionDuration: "1d" - batchSize: 10 - batchTimeMs: 1000 - ``` - -## Configure it with Function Mesh - -You can submit a [CustomResourceDefinitions (CRD)](https://kubernetes.io/docs/concepts/extend-kubernetes/api-extension/custom-resources/) to create a Cloud Storage sink connector. Using CRD makes Function Mesh naturally integrate with the Kubernetes ecosystem. For more information about Pulsar sink CRD configurations, see [here](https://functionmesh.io/docs/connectors/io-crd-config/sink-crd-config). - -You can define a CRD file (YAML) to set the properties as below. - -```yaml -apiVersion: compute.functionmesh.io/v1alpha1 -kind: Sink -metadata: - name: cloud-storage-sink-sample -spec: - image: streamnative/pulsar-io-cloud-storage:{{connector:version}} - className: org.apache.pulsar.io.jcloud.sink.CloudStorageGenericRecordSink - replicas: 1 - maxReplicas: 1 - input: - topic: persistent://public/default/user-avro-topic - typeClassName: “[B” - sinkConfig: - provider: "aws-s3", - accessKeyId: "accessKeyId" - secretAccessKey: "secretAccessKey" - role: "none" - roleSessionName: "none" - bucket: "testBucket" - region: "local" - endpoint: "us-standard" - formatType: "parquet" - partitionerType: "time" - timePartitionPattern: "yyyy-MM-dd" - timePartitionDuration: "1d" - batchSize: 10 - batchTimeMs: 1000 - pulsar: - pulsarConfig: "test-pulsar-sink-config" - resources: - limits: - cpu: "0.2" - memory: 1.1G - requests: - cpu: "0.1" - memory: 1G - java: - jar: connectors/pulsar-io-cloud-storage-{{connector:version}}.nar - clusterName: test-pulsar -``` - -# How to use - -You can use the Cloud Storage sink connector with Function Worker or Function Mesh. - -## Use it with Function Worker - -You can use the Cloud Storage sink connector as a non built-in connector or a built-in connector. - -### Use it as non built-in connector - -If you already have a Pulsar cluster, you can use the Cloud Storage sink connector as a non built-in connector directly. - -This example shows how to create an Cloud Storage sink connector on a Pulsar cluster using the [`pulsar-admin sinks create`](https://pulsar.apache.org/tools/pulsar-admin/2.11.0-SNAPSHOT/#-em-create-em--30) command. - -``` -PULSAR_HOME/bin/pulsar-admin sinks create \ ---archive pulsar-io-cloud-storage-{{connector:version}}.nar \ ---sink-config-file cloud-storage-sink-config.yaml \ ---classname org.apache.pulsar.io.jcloud.sink.CloudStorageGenericRecordSink \ ---name cloud-storage-sink -``` - -### Use it as built-in connector - -You can make the Cloud Storage sink connector as a built-in connector and use it on a standalone cluster or on-premises cluster. - -#### Standalone cluster - -This example describes how to use the Cloud Storage sink connector to export data from Pulsar topics to cloud storage (such as AWS S3 and Google GCS) in standalone mode. - -1. Prepare the AWS Cloud Storage service. In this example, we use `Cloud Storagemock` as an example. - - - ``` - docker pull apachepulsar/s3mock:latest - docker run -p 9090:9090 -e initialBuckets=pulsar-integtest apachepulsar/s3mock:latest - ``` - -2. Put the `pulsar-io-cloud-storage-2.5.1.nar` in the Pulsar connector catalog. - - ``` - cp pulsar-io-cloud-storage-2.5.1.nar $PULSAR_HOME/connectors/pulsar-io-cloud-storage-2.5.1.nar - ``` - -3. Start Pulsar in the standalone mode. - - ``` - $PULSAR_HOME/bin/pulsar standalone - ``` - -4. Run the Cloud Storage sink connector locally. - - ``` - $PULSAR_HOME/bin/pulsar-admin sink localrun --sink-config-file cloud-storage-sink-config.yaml - ``` - -5. Send Pulsar messages. Currently, only Avro or JSON schema is supported. - - ```java - try ( - PulsarClient pulsarClient = PulsarClient.builder() - .serviceUrl("pulsar://localhost:6650") - .build(); - Producer producer = pulsarClient.newProducer(Schema.AVRO(TestRecord.class)) - .topic("public/default/test-parquet-avro") - .create(); - ) { - List testRecords = Arrays.asList( - new TestRecord("key1", 1, null), - new TestRecord("key2", 1, new TestRecord.TestSubRecord("aaa")) - ); - for (TestRecord record : testRecords) { - producer.send(record); - } - } - ``` - -6. Validate Cloud Storage data. - - To get the path, you can use the [jclould](https://jclouds.apache.org/start/install/) to verify the file, as shown below. - - ```java - Properties overrides = new Properties(); - overrides.put(“jclouds.s3.virtual-host-buckets”, “false”); - BlobStoreContext blobStoreContext = ContextBuilder.newBuilder(“aws-s3”) - .credentials( - “accessKeyId”, - “secretAccessKey” - ) - .endpoint(“http://localhost:9090”) // replace to s3mock url - .overrides(overrides) - .buildView(BlobStoreContext.class); - BlobStore blobStore = blobStoreContext.getBlobStore(); - final long sequenceId = FunctionCommon.getSequenceId(message.getMessageId()); - final String path = “public/default/test-parquet-avro” + File.separator + “2020-09-14" + File.separator + sequenceId + “.parquet”; - final boolean blobExists = blobStore.blobExists(“testBucket”, path); - Assert.assertTrue(“the sink record does not exist”, blobExists); - ``` - You can find the data in your `testBucket` bucket. The path is something like `public/default/test-parquet-avro/2020-09-14/1234.parquet`. - The path consists of three parts, the basic part of the topic, partition information, and format suffix. - - - Basic part of topic: `public/default/test-parquet-avro/` - This part consists of the name of the tenant, namespace, and the input topic. - - Partition information: `2020-09-14/${messageSequenceId}` - The date is generated based on the `partitionerType` parameter in the configuration. And the `${messageSequenceId}` is generated by `FunctionCommon.getSequenceId(message.getMessageId())`. - - Format suffix: `.parquet` - This part is generated based on the `formatType` parameter in the configuration. - -#### On-premises cluster - -This example explains how to create a Cloud Storage sink connector in an on-premises cluster. - -1. Copy the NAR package of the Cloud Storage connector to the Pulsar connectors directory. - - ``` - cp pulsar-io-cloud-storage-{{connector:version}}.nar $PULSAR_HOME/connectors/pulsar-io-cloud-storage-{{connector:version}}.nar - ``` - -2. Reload all [built-in connectors](https://pulsar.apache.org/docs/en/next/io-connectors/). - - ``` - PULSAR_HOME/bin/pulsar-admin sinks reload - ``` - -3. Check whether the Cloud Storage sink connector is available on the list or not. - - ``` - PULSAR_HOME/bin/pulsar-admin sinks available-sinks - ``` - -4. Create a Cloud Storage connector on a Pulsar cluster using the [`pulsar-admin sinks create`](https://pulsar.apache.org/tools/pulsar-admin/2.11.0-SNAPSHOT/#-em-create-em--30) command. - - ``` - PULSAR_HOME/bin/pulsar-admin sinks create \ - --sink-type cloud-storage \ - --sink-config-file cloud-storage-sink-config.yaml \ - --name cloud-storage-sink - ``` - -## Use it with Function Mesh - -This example demonstrates how to create Cloud Storage sink connector through Function Mesh. - -### Prerequisites - -- Create and connect to a [Kubernetes cluster](https://kubernetes.io/). - -- Create a [Pulsar cluster](https://pulsar.apache.org/docs/en/kubernetes-helm/) in the Kubernetes cluster. - -- [Install the Function Mesh Operator and CRD](https://functionmesh.io/docs/install-function-mesh/) into the Kubernetes cluster. - -### Steps - -1. Define the Cloud Storage sink connector with a YAML file and save it as `sink-sample.yaml`. - - This example shows how to publish the Cloud Storage sink connector to Function Mesh with a Docker image. - - ```yaml - apiVersion: compute.functionmesh.io/v1alpha1 - kind: Sink - metadata: - name: cloud-storage-sink-sample - spec: - image: streamnative/pulsar-io-cloud-storage:{{connector:version}} - className: org.apache.pulsar.io.jcloud.sink.CloudStorageGenericRecordSink - replicas: 1 - maxReplicas: 1 - input: - topic: persistent://public/default/user-avro-topic - typeClassName: “[B” - sinkConfig: - provider: "aws-s3", - accessKeyId: "accessKeyId" - secretAccessKey: "secretAccessKey" - role: "none" - roleSessionName: "none" - bucket: "testBucket" - region: "local" - endpoint: "us-standard" - formatType: "parquet" - partitionerType: "time" - timePartitionPattern: "yyyy-MM-dd" - timePartitionDuration: "1d" - batchSize: 10 - batchTimeMs: 1000 - pulsar: - pulsarConfig: "test-pulsar-sink-config" - resources: - limits: - cpu: "0.2" - memory: 1.1G - requests: - cpu: "0.1" - memory: 1G - java: - jar: connectors/pulsar-io-cloud-storage-{{connector:version}}.nar - clusterName: test-pulsar - ``` - -2. Apply the YAML file to create the Cloud Storage sink connector. - - **Input** - - ``` - kubectl apply -f - ``` - - **Output** - - ``` - sink.compute.functionmesh.io/cloud-storage-sink-sample created - ``` - -3. Check whether the Cloud Storage sink connector is created successfully. - - **Input** - - ``` - kubectl get all - ``` - - **Output** - - ``` - NAME READY STATUS RESTARTS AGE - pod/cloud-storage-sink-sample-0 1/1 Running 0 77s - ``` - - After that, you can produce and consume messages using the Cloud Storage sink connector between Pulsar and your cloud storage provider. diff --git a/docs/cloud-storage-sink.png b/docs/cloud-storage-sink.png deleted file mode 100644 index 6ccaed2c566eb5f690f8568ee653471f7afc4775..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 19196 zcmdS=cVAOq&^`(WL=aIC6p@b7M0$-70Tt;GqzEEadJiphLQ#<-E%YYRL7LKgL}{V- zNR9LoAkqT_IGgYN+`s31&MP=4f5P4~v(}z9Ysz(nzk({=yv}$X1OnYuQGTfn0$mXX zfk70WX2^ViiU*U`Se(<-?oymwg?!4-U0wN%|_ zDC0*0P0uR=pZqrDP?gM;Jq)>hw}2Zs6=oU?*JL$d8KP^#pUC*H4y>Qy-O z&MrDPP>bV%8a$B~EW5@gzj=FKon1WlJqCfErL?OrfP?zt$*qrfmp@V5G_Kj4xT!-L z!5@zi?y0;r@`x1V1~q@M!}Ls0d2twv4fHs`&Sz(5e`1N!khfayNwl`K%*%d=AI=)J z-ANAkb0Hue1`1M-f1^Ysk7XKHG>?-joqqM6Ufg+canbtpaLd}wZA4u|BUQWDa!Nr~ zO;$(NuvNabBvnxxkxo=RE;~880*X}x7SON&_AMc|xW!vx(lX`X@3}R#ETOFOp!Z=f zXD@eeQ8FZFGYJc|xCT;YS^)1bb<;K23OeWVD5)%ksWioOd%SgNS4uKQygxUNUXTc_ zblyK91+i0xVFz9i$t%A#zGqv~bb90NZ4m0Q{3z_kfprv-``GCqw9JYH-H3!2U^qaa z*_T9a#Fd6cO|2)TUT*riU>TXR>Cb&gEMuWa0=!F$1@x?r1~nUoeHMDG!>u^5{-sM% zNHwGL)kJ=&PRkb(kS))Gmw!8qox_r+Orr0pv~(JJe%Uh)&Vf^|sQcR8nE!wbRL!*0 zBmq40(O$xzPp|Z!U}$=3>hLM#VK-m~Hx62%$@l{Ftm!stI+AcR?^#)>W~eR?H@CLL zsq#p7V?6x6#8VCVYamR5TG9c#f2fo2yc3~yevDZK;zyp530~9Bxr!lM^GtCj96<~Yu4^38v z>su6{ZeT5yw^6}@ZyG(;m5g*AlhYr$MmyrW8)F*ZysAxggtW+zfR3LLp@IwGq37o; zR`L+cpFj7jkep$TRFmAk5S2KmqB=eh`*av~{TY#b>d{6~)D0h~hQ|6CHRXfpP6_`0 zVP;S_#e$bK4J!DaLMC6Yi@P-0?-AcqwB+*4TTr*)f>$H(FK&K=O3Qa}9)=0Xz{Yq< z|NegU++eIrWxwV0`WEl>o7$SWb5M}SLxc_04pW#)D339?nvsnSb4sVJSerdpURlO~ zBoEr>K-kdlFcm%DCW91(W?1Rn7)b2DHJKEh3iWiSHv=gPEqEQ$ph!Ml9*YHbZGGD-S5$2AaIfk?i?)WAuPxcP0*%4*KlnitJBxe_h4>gDdr z4&tQVVIo-oyLF5c(Bc6ZLg}WJTP!*fBiBfuh1^DE)1cz0$~$2m9toY)CM>ay&n_j( z6BX|+fJL1qE3GJSoiIm7bv#A4RPmignuNw&+#S$YNEkNkJt3-ZhY96Fm)=RO#T@lD zn}#fCn*)(c(+;x?b@-FoNqz8y@p))2$=Ap*EZ|g3#jEy2Zf{%Ed<*0O?BmumqT&X@Yb}3A-msqWxpUH0(3TQW5dxUXSPt1AA!NU9 zq*;nafrd=lVIHJ_;8$fCzdaQa8`^?BUHZi2N^xwUuTsGOuJ15~>-B%rtmJ#-wWa`R z*e3A7p%{74Gm9|n8s`F-*5NW%P1Csp;YS={3n|vouG?(nhHGM!K+m4G!|IN%)9tP4 zb{Ji}oHY9P3{A8(L}xUXJvFRlB^;T48#Fy}LC-wV5{*IW4YOTg1GMGZ@EiOv|Csva z%LJU>ma5oLyQ@?&HdO=km9iawj5xLKC_RNw@;6o6$|8Kxd|6L&Hxqqh-UrIb7>Rco zzZOM#e|I$TOa9QQNK`cM{`f{-UY>fI;b3Vgue|D+p$bbCE@SMu;B~SfLKv1peY-5> zPK8`-b#;k<&fD|j+8m!_C)!jHV25y#;0Y7psBpTqYsC(pv+<2ZI3Cc?D1uX;8tiqS z3b>D3(D!Bja98by1T=i^~9WD5^;o`TI8|1#19uP=KiOiGKFjxS_h}0UM>M~vC=k}=L_J*7g=Dc zeeKwv%9%_4Z$V)Ni*-ivv4xR`rU8ZK(N8M=a@~u%tKvysJ_Ca`XFt_Xif&!a@P&-JVq{RKHv z0eijd3Ec^TB=v9*w53L6Oo&Hc}dps z8L>-{YqG^{81X)jDhI1 zDL*czzXq~hZ(+jm2(F7U1a&t(?ZUg`%L2=K=?4VT#7b0p_xudi_`mq(nj_r&(9Dgs zhh9BcFX#&73l8@V@YZXAYd+;U=&119gLA174S(k=4a6CtuhMuw8NCs{@K!=MqZNW# z1LFBZb4dOM0n$x1K4>nEEGML^SqB!k?C3)EPbVa5Y`_o?r?#g%Z(HNxz^1r1Iu%FCc41hwY+6mC%1cWxR}orxw7e}x{?ZR z+B1|6CPYOS=@q;uFEdcnD>q6lF;Gj}iBwWuxq|x@uE^6Hp1vnD*gU3EoUR9al0$am zEE0C1%6q6R{}%IsDFab#Ypty;T#>NP@3TE^adnT)i>{wu^5vkU=Tv5k)SB0B)V$TL zo@AkKf)}a7fiXJJdTW%Az;-W)iiQG)wVP&5FQTHN)}=pvQ?1E-;-`-$B9fJib9zwQ>Uf%3Jr$N3 zTY^5fr{e8C&EuMyQU#y9nwlv!SbyQhk95+F%|--FQPymH^Agvi3l}AH(~K`9`g3@X6m@c6;Kqo$MSoJ((mxN;fahPwtc+14>?I?NsKFVnLq$d-4$8wtIhm5R z%+LePx_kZWkjEwz0SB2}?lZ5s_8Q_Q9jrEwjCYhp124SFsj^tXE^zeBkv8#z#gQT1 zUWHeJJ(OMm=kYrLZA8Ji!z3`Q1iQTKSbAH~>k-F8CoaEY+jsx+p`~}nZfe^|MOTf}X_hX8H} z*Uz=>T=6WWGfyE4v^BSu$C}025xI6JYd2TonGRX~0z?uXi5nukoZ9@?BXW-Cwz&2d zx47JQCvuK)B_&bEq5mAxc}<&3&8(;Vljp_%KDrgutYSe-ViF@Sgf#S2s!`3^=@j8l zcychy3sq`!E!HDJgs9N`gEbBC*=+4Zg`dxb!PUJ*7OtwZj!K^Z(~dOAQ*b6&?8as&W$^AadW? zJ@p+GK-&5rG~SC3m$+KBP-`x)6wjF&_i`mRk|m42uKUQDH_c1OZ9m`m39xZOBll67 z%?L>!S9pDe|A(b~10-SnfcN4Yoo0WrSmv>R1xQac*jiG7TuTDZv_-^#>E=(a$)RiyR{66g2w+s{5F*H*svX=VL$A*l-t!k4lZD<28ORiBnY)rT8HpL`^ zbK3UQw4JEtDbD~k56HC#Q!bG9Vs)1uCf&|aZo8e9665?RW%Y1n^-*$bQ^lB13mfR! zRGtLlV$^9njwDr^VXMabY_mrA&{x*J`m1Tun|9bsMn*>IxpHPPF|o$;rHP}o4AJ9J z`g?sPcn{gHW7CHOs(d)idu!NWIMChlcK+DBn zN@+~ICrigy_pDW|IqVX@AmbuQno6cJ_7ty#bTMsfbiotbi)>RxkHqf9ogyQz>VJ&~QtyoKbskn|Egas?*i{~Hj z14-opSMrhxpITb;6U1p`%k**9tf2_rklv)gcUc}s#gggBu96o$%_7_)_cb8T-4Ir- zV4cjsz>v*ZU&IjfLZMb9Tpgy&bkFfqUMDQoNqrlVqiO0!N1HN0=y*R5H!dNbZlRm+ zg%hHr+!%vc!U_rt;Q{D;_)&5*lk7$`qU<52rPS}MWsi(wAlJ^xa=g`Elf&;`{R;oH z#vJg2XA>$cb-1U6zpAlFF135N-i|g-DMWu&xvhFqEO;&6iFXLX6Yn-KJN5wbX$*+G2Jz6o;hWes0P%ggk z2Z-NdDIxQ#&c1c8L;{$YGKq`Fj(c1K%g36mk~4UeOKfJT|c#CazD#pqJ{yK?eS&o!iXrAH_i~Oe27a+uC!vqvug!xqy0>%^!c5~g9M?>7IohoPk*|}&#nT5$VgGLS@2_wPmFQPMJ6PI=87Dg{%eI{Gn5&BB(wyiWzjdzDmI0Wq0X#j!J~QK% z$mhv7Tfe)D5x$03mt#Vk+)bOFgkfXnSK~vj$5sQ{{_L4__-{75d}=i@@K9;n6ymbz znt;xY7`-_B$%j_!p3=A}pOtq+-OhuNSsU#He*2G=I?S~d7XGMua{Yp zhYbf@?t40uF#>j69HOQSa7~y%%xQ)Cz{j(8d`6WGh3d#Z_lgWp*qzKX;L=&lUy|(B z!AmA5u)sB71tHtqe)rST&1G1EKc~vwtWgSZV`_^f{Zg zWL^91UpTV>HCw_uuxfOMN$)_q$Za%_Qc6HRSv1OuRrE2TkllkGQ)7euZ4}ZHi)spQ zFMa!EE`Hy1f-yjcf0Ab(5pqKGYwjykl1jXGT%gHg5wU!zYEL}h)|l6c$?NW>LaQ}) z->~$Bs2-0TqwjxwfLMKVS*9I!4mKHPIXJKUP-9iwUQ_#9H~w--4lGi(dsNG{sI0~_ zc^b5N$5-xoz+s$OLrS`X2|N5}>-Q{XOD|uv|RxN%-+)^>NE^wYC1@ zn-X%k^A5`0en@&*-dT4-g+gEKFy>4A%x>Gk{D5DETl;cHyUA%zSz}4KR*J;cy)wgy zl?zAS)C;&FK2lxh%rbp7FS!vW5t6Yn!Y*_2%hBSRvhjBud9Lu*Xg`4k)6>5_uuWaZ znOhav86_vvbn`%R3sPwNm@?%cQ8v#5HzIYb*D0aD$mv(U4^=b3l)#n zKQsUgZfRry42G6(|Ls9d^NiB-h+zFXuH-r#9?Bf{%wZsaz)6-E=-~Z?8nTr$8@fK9 zCSTN|WXUVz{f^`hf%wM00sp@fj z<7r0_uk7FuJF${^P8>+x2%Y;=7OgZvn^F;-j~^4+>a*VVijN%$C+plcL$_uh(SHyy zZrtaXkXGvxFIQ%jnJpe;G0wVe3SRUwDs%=`?7x$7c4DPmOa~s|7T#0*1BC_hJCpA~ zY{<)~U|w5O`8at)D}i8!0DcCD6Q7Asp*cb>e>(3D<>$>Sz{UHqQ-uG&ibm)9m^p^7 z7}6;Fpw+n(Uk{zV2Q_PTHw1Q>@B_vZjv2{5_yw9$yZL{3oeeTSb>Hd+X(a@ z8^godvUTQ<>n}-ve+-JM!plE6ItAJPz*GJ{V@$(r5m@wvNiC8s%54J;qvoCj_<5hI zs*o;tMei?kuj707TO6oX0$3h5#!tsp7MOXqCd6&#Z;X3Egsoel^KLw%+ zLof{xZ@xoVXVawC#=Rb|bksdw$-kAzV^WcFY9g3kHkjNWYJ8cXBMm$QhW$5)_90cB z!^6%w%?4$JWJv`!Kpj7wclt9Qeb;+oWdzGsV*eTi$G#vF~3)&X?u#K@&gIu=J@pb05^9j3|oFsxj1~Hz^&PRU4|}qlh5~}jl@pf%+gYxhfccf`vvP~`|j>Ygp)2M^wcdN znW&Mqk$)%UZ-UsF(K94NX0!4;GVW6(D@w>$*#G9|1feg<85{?;eKB z{wMq+C#Cd**d;|1CZmdn7gJde0_!T?(MShVw)Pe%c^Ofv3zObSR9>e^<9rxwXFRg# zRcs^@VAB*caj{u$DZsbUbT;|VbDvfJLSw3kaR)--aU}&lxLGVNbNc7U_`q?~$IZ{e zW21dF!WYG9_f5pKBojZEax?xUj3oV34#`mxTnY(TCsNS=QX5uLz8jF13|0TY7d)IR zeB!<|CI$p`BZ5|1vDmWnEDHahuur`_=AoH5)T>aU{^Yo5Bra_ka0h&1qF6ashDO8SyUR{J!Jav zu-6myS|rPmZO-$#kF%y|-2*@`()Ug&^ofoBBFFXTlZXBTmGsy-1`F8$gm=IgAD@_; zFOW90owxOWw^uVH<;bCpDAF#88dFi^p!}R*mRC4Q#=Whq1$1{WmHG9;W|g+zckzLf z`}WmEA#FZhZC9$m>`_ymaxF3^U!g#OBqkV@8ExH6(CZ-1OU*}nW;_&g+Y(x?b2av{ zSzHv%s;!zrxG4rwu^5{JS>YWwG+?1&Zn+&))dLz+PHgTIQ7SPCF%O#qV zU8poy2g^dFDk(k-!E{^DXa4M7oyNP<>66}3AKp|oH>%3wqM=kT)fP)t!O7MguOkhb zTMyS&`ei+3FeSj0!S~?Lfl+SE<(%WfRe!iC)r#>p;1gvx&18g7Iy=_4Mp-uMmxm&$ z8q!3iq5~%LAG%j0oywzyuiM^})&p+OK&rzkp+^D;oFn4{O%96tUXmK9OYlmdPHY%=IV# zYl4Z4r_8Fqkw9R8R$FOfnXPjGcMApERZ8{Fe4lG?i)-fT%%0L6a>Fxyy}v1R{OXOx?^Srnz&)cQ=fpXC{7V+8cHOcUz#;W z-By|ena77#HZxCrHDa!Ylf^g};#0Q1<7hrAt{wTJe@7ECeSn*|D3r*AiKf3rBtI2f zI<{ULwA(+j1hUh`kvY(S?)0au+3BgG$ON_-R#Y~+iRsJbG5&2s4-DV(D|YgLz|!v& z&Pm%=Y$jSsn^C)*bLVPP_a5?z^2mem7+gVeo1X*M{aslDauxs0!PujEWO5LHI;Zrm zNkF6FNbJv-NogvP0{y2(khq^8*8l&yr$Dkzb>`41Z4IM|50Kudq<9!#zHdRPbNqvq zz&{h-BZM=^0(eE`IdGr6okm(KyP5Ius-WgL!QBcJG{7Sp7duHn%@{n2@lLz19VJBJB%&`;ydYW!}t zJ(-!6N!U6A#?B5_RmYQ+4TQMLV^;|Vsh!#!u-e7!AI@)N`j^{Kd<5mxZ=AwA0fDm8E^i!Mn{1ztSsLn^13o5m8lH((md*nR{ zG*@FV`)vkEq%>e4{~nr{L0GIB=%5@0*osBOTc$ zp+IYPfF$*xeXd(aN2ih1<0bgce=Xx-rQW$H!BHr;sP}GEM8nQ{rIeK++TW_w_7*TP z^Eb;IN@AEC(VLi~mwP6zdU(|jYwHpEwfO;1_9&aZ#gq_c1v!oq`WC8wXCD6Dp;@Kx zuf0!EtZepcf3^Kg#xT(EaH7ezyuEhOj@ntf4nV2h)zE}Jl6*YN^`ChIW zR!G0o#G>ashlhV&OmN2wtkn(vmbi551tP1#;E9ZuLaiy12IX#kIY#)??WSAGB%S)x ze{D15IL1$HW%h+T1W}BOJ@yUig!OEqhm;>DhNcanu1Z(OjdZjGSH6c0a|p<@4!7WQDf7{HPAjE-J< ze5#*V)TE~Gj~w0p1N~3Y1fb|K4s&pJzn(k}w;AniQYbaxNh5dwTf|@>&ABX@ACQP;pY@8tFL5`4F-CQph>9=KIIgAF{RC zX086Ohl#dI=u2!s(|4Jh))90tAALuGRO&M73qjSz{CLV6;5PfuUuf*CtraV@e{#To zm-F@A9%}}cY2DILVJ9t-si167Tk2sdYKuqrJ{Wvq?fX?bk7CH+I zp5x@^4jqZD3MCxEgWJU)T20RrSJUU;-a9jD>EfAtwOjkik$zHEYuZ_p6kiOKIFz$H zYvc_S;T{X`cjxVo_m8i-|53PFML>oe_x47vbeBZVjdk(B_)-IbnKKjX<8%mj?_59q z=!@;&m0QP^20r>|{>CRWB!$&_fYS{9&l=3lU;IDUfVK@O$tAw;pZ6pd4R=<=4-dk1 zN`SD~kfb1|kNZ&?s8Jh_JLr$5h12dpc)l?a*!?WLEFO0rMbPa{HO)U%vaJ-J-N~(Z z-JG{9M*jgan{b#YCM}+s`z1u{kfsI8?l>gP7UP$aY{9F2_>t& zqVyJ4`e|SooDwgci=$zTPa?2LknZ8VLZ`#;l=FUw-tY2ZCi-9 z3NdOkU6tn&lhASuk3cZcIPEj*u@+Ui-f;h-cJ&+N)O*;U*zBSt7p{P$ma@_;=Q`QE zdUdqU^)R8OBATur?9A_bG+^LviZ;5?opdZdmrQ-2VeRRMfOTy8RG3? zYs#ztRJ*pdp|wSfuBwg^+Z=e-)j}du4is;v~*4>8WYeI zcjdc!W~^PS*?Eznr}-fMg9=JN;whHjfZBJqzUQyRJpN{cBlV=<$?;cagtK>E0S4%9 zRQVt8{@(ietC($gDKx$W&zl;PiBrDoYh>vOPQ5l?Z%WsHOr6}nV$-{V2J@{P1kk}E z)>ZeM-Z`)77+GS6PN{Nke$%?&uqe%*C-?_0zY$R`XKR$UaP)8BU)*_pOyHUMDrpK1 zQU`Xi*9A_Y=teH;@(@BaCQ5KeT|WHNVk&?cc@P$r*4y9d^A?5ZR|35phW)=r9eua^ zR?3?~{kd;o!n5zDg8RUGF89IhN-$+e#*_kbgi#RWIQ#SG&&Pf&q8tqSeHPkE9xcz^ zfjzP!%|f-5?l5#Bgjv7Pm1G)WQBYW zpOmoLSP?RrXwvCeD#${X`un*L8=ZSk5Kc>sSOt-zGei%=^(>W<9Oq#IpM={@vn&Mighs7`4&O&lq?6$ETsf6C*dyS%!^l0<^2(?8m@(Onz<&%}doZsA_2S^j z_4MVF^QTciuNe$hwq}A$t62l98fTH-d#b>-&MR}(-lPR6?K3jK*Bc zZG>y>@Ph95xmPR#1>R1<0m&}BBXOC)SiXdrfou9LkI)QVlNbPl$U@|rhnBeF|M01; zr0}V(FzF}tQ(7ogoqs|4xHkGde}$xDaNRjSU;`tcR;T^!w(Tr7>|GgiLhP^|nJ36k zoSyjgtQd~JRQR6&T$yR<$HlM%DI_2FsrJFyb}MYt%NiXj%6YgAqurJg6{>*XI?0AA zlmDcgS$9P8-~6R>4figO^Xx8t^?-=6D5k|Dw)*v9^pJ(hkmI>buDueQo`RnT{Ne8g*#?iRwk$OF7!wlt^xa$o$a?OO8#`%_e^&WGVuSlP7jQ zMY!xfmY)jE#s@YmM15F$nR{nBGWlMgyB~v-3M4cuUPZd2H%S1Jg3Nf4gY>tH#@co8 zTK@N@9Ybi+*mEe2-`~VahE|m2`R;DWai+xl%iP)JDge=NjN%o!>ZT}6yur=~4Be|f z(4@>g07mLkBi2^c;q|-|=k}HV_P<0l986Y9wWO8$ul@(uiCCc?q~y7u`B=2ly^VuY z&>#nf;DVdtsX?*#R6b)3fGz-2JM7716{7A zx5RmGNL40)ABPv(QDH*we9TLv29n0W)+S36-^e_!l04`%+-ReDjvB=7B+Y=L>Aa>6 z;CX3^-#)li$@gidy3`n$s$~ds=KFidw-HZexbRbqmg!@5P0?zZm{Sr^o(9h%_?NO- zBY;*(BeyA(IC$I@LB}`cslN%N3vTvPXhSvY={>7Glel&+>O4WU%vwg&JL3xy_A+TY zKpn5;tO8Nd_{98YcsQkTOd2we%zD_i?dcx3irSpAaV<`g{(__%3GIM=5Fs5kZils- zV|km5bwm%PgO3TJAyS8-ISmI5m8tW;h(_8vLeV&1YLFcUKphCvrffe%aNN7NHR+z1 z&!Qvh2?;)Ksk%PrXw8S`PvyZDc^}0*N30jq=l|bc0yd6y0nBKwiT)Fv`-DZqY~pSo z&wxORkmKXDJDrg$A=rfeJwFu#m!N?C&|-QLOM@n0{r7{{FW<_ES*Z3E+&Z0jLK{OK z$@$zZFh$7pwK%ph-xdL!E5U$L$8Eoc|B%5~olj;AqUS)}m7ZO&4{<*|0MxjK-Kthb z7FJ(0&`Zg~z{d+s^tM4~_ykU79KD+SEUPW4|I|bhETePi{D`+OSOvN3oNcaZb+-OY z{8CW5XD6%=++5{&%RuK^rYtvr83jO=e$20&=oi3gzqFY1Sd?I(C7PY4GXTWYhjn~B zzyu%W5Z=Wv94J&RGbmcT1-^vZz83RJqS9Ms>TF%M{>+{qaM@7#s95 zPTC%0usub^9s@jf+DW&KGimjBKAhndh&I|cLo<7gJO0X1vXXVT9`3u#@X?NJhsjCe z61Q3Y>asc=APHxOkCvBcd%*+9gPPe9bt`ueqUJo0F3vJ?Qop2TrDPCGTu3>VJ9$*h z767#2);GHKulAEHx{n2~y`){Ix*$sX#g-3(V(HuQ6KkWt@`&DQ&ry9dN61s%Dkg4a zvjqUNXYpTB`}iI`*+bX;kAd1Ya2Y_^0|akr{QiGfEn9$n(e~7CJY;OI#tjKv-J7Xh)nhUgfq1Vp288c*BT5omI~zi z>_eR6r3;zTVSW(6xQ5E4U75e#mB|b7|R7mBQJsG z)&e9!CxC3fcd!osfcVpbq(f8y0A+O=jBOqRhzy#q`DVWxlSpOVo6Q+aPyAmTJg{Nh zih`SRmSqwUgBt&{ksO0SO>}ch)@bLY@vBJyGsFeR=iW6vghHVkjn}Ia$3&V2R9F)J zS3;BtJAy;RXmwN+^;BA6%0GdEI)P=Xw8J>(zBWE`0g5h+ z#aCYKMYiq{mFWtluiW{-z`k)%kbM7@2wV4tYbS2u+ZV0CGgW0%d;K=p?*$3R-`iOz%#k=*`<7- zG8b9^|1GiJ;#tppXKDFi&t6Qqw%F0Ir6vE?nlA}RS#rS(aT{RlEcYg}$rcTiB+jFY zi+Nk9ALLsaZalD=T&diG2Y^6%S9Tz$09Wd!+dO7yXy~^+#XDu`D@O7Im1PZos6it! zgiPS(VGk|Zv$sl%kqsi~<9Ef?8JmWJJu!_z7ej`A(WB%nz}B9sCkRXxyBa<|mG5t{GEci-8w z@6|6YCQ!H3f|nb>i4qK#uG0&B3OK@kYxu2r_%@?+l2dy_B`65A12F&$PJZoG{vL2% zJts#cYZt#K^9uUH6SnD|1HqPRdJ4f<9)WywB8((J|S2|i0 zs#*Kd@lH!72=tu>)f7Xx$@zJPasqy(?CXPs+?yE|(@zb&YNG#iH?BH2gMt)kP>}y0 zSxvb9O{{_DLOh%f?#gx(Aov)EVYdM4lXj7w{39!XzcjzjXgS+0J%Wo0r3MYD0C5-K znB9oY9Fu@sys}h=`}@zR1EEur-OG}tyD^N>I_V(}dHJYaJDvb6OYxYV>K2Yg8Xxs8 zLvc{(x6up*XiajVjtQUy#%)CV$IfWgPW-E$y>IQB@)n_!p{O|#SITE#<10f78q#Qo z0qn=CRZ4K;@qj#de8kGqvc2iPv%US}*Hziwl|7=pmA$YbRIiq2uk^g_O%QwW{}2dw z3s9&)t(neOJpli#C71m&7iUb#i?xcgLR!UJjrFo+Uv5DwAZ$z6b$xtnYXW3x_-RqMKJ0FYYdCdl6{w!rV zyZo@XS}RE_2_8!h$a1Hp3de%}#*-@wro}VJg@n3(?VLH-dVKEXeKXU)PHvntiUkA` zQSReJTnVbY%)Lj1uXbdSit6UkP%(xkmv9+~TW1J;W|G<=ES?GL0 z$n$it?Av*hGOCb->g;IH;p=;Rg1H=9vbgT}AR+MF#=Dol*HoZOpy2x$&F{Z&C1ddwRCvfH9*M#ok$^@N%oEsf&9$?1dwF|86+4USqnVaI740Z{BPpGZ z1XmvuRI66`OD=-bM$@?|#Qe?e$YrrDLQ!BO(K&qZywdxJDLqasWc9`L^PLm=Hc!e{ zQ}f!340p)Y-i5%f=0Ac8b4=sl?`WHDuXQZrr#LN|S+~P`HrobBU6oQ%A+t|Zx*C%~MEYQAHbTHrw54XhcpX1Y% zD3abP$U2Zjk%RKIb#Ej*u|k!@5H@wiS76U)N?%+0-1}hB^CJ8;EzF*aPPS zHyq}=!Fg-gOSR;;XCmx6%n-Bw1#oHGwa_#FF|Autzi50j^AB8OJTAJ;HSfu4zuZv^3>E$2;Vbx$@}ysILk!#(>F1}e_L=@Bs|NG?~M zz}6B%7E>aBABU?b^x^gr%&KGH7S;ID&* zFlrFfR7-=$hQq_|2HK>8#TgR5&W~jhi*7YL^M`OekhXoA~F4R;0-#zrDv$X z=Pz@zWq;%PHmr^;PPz5F{+#z~8BvhDf7rX8<-2vj7BYw{VL@@I#eycWLRrgi58raV zxwY=FhB{U_dFtkCSQPm8El)Z8RSEWi5q?ch4=ei+;9; zwHMotWczX7o@Fr}R@0{VM17Do#cnj=8hyt^-NgyJO_yPDTQ9pM+x)?_~) z7!0(2;`7-_9-h9p>oq%g+jsivL8qLVp#y|uvXBuv`>6M-n>$4F`}_6W=Z*8%*D{~} zncrO9T+OZbyp$*&f(Uw($E2oM7=|cB z2nM^t$KHnyBIcYeC0a&%+kz!;Z3 z{PAPLW_@`iJh1@t{%)qxpZ!X5T$xa_24`C=oL>6p`i9ycm4t1jdPK#!(Yl<#cO?C> zvw`HXQ861L>zAQk7c6OTH=LWxG==0*2%LFs*lHT&7q^N z3B@ZuxvfLP3BC&8U1>@rYWkv%>4j1*jX_Qb4T#-<*~!n6Sh?ci}0)u6G zO>9LsYtf#aTBUBbhHnq9IoSjh^nK-~P&6*=^iwcdEUJj(vbVg|T`J2L9G8i%k)O=H z;rr=2ql$(}g~@8v*qY*_ZD=7=h%XnRHE6cG+iPQRFR+gO{yyyUW38KFAanRKevyB$4?bd$nUkBqd)|QQoctnPZ}bG zC;wb)RT<(5AE7@t&i)|yjd1p9h}R_gsAnx4Ds6SKsy5jw}MN?g~<^zMpYl`ua!4N>G0G`OcCyA}uC)yrmu3>_Gpu7uaMddk*ptOkL(9g9``&?sPA8*Y~JVt`nvGn`- z%;+~R0=>-7eJBO1;LXAV66f$#3);ZO28HVPvXhohFF6rP2)GiW}1r?`i4-#i*cl`G&CQ~BJ5j&qSCfwrOuGJe+w)rhOI2- zS$C&OalC3_>7*`da1FjYUUCFTj^yC1{1it_ij9!C(Q5w{V0YdbdU8~^g z=)_xHWp|0*5j7;+{HvaWmcnd194j_k4?&H=V=(v%7#bhm`;z8RNn=v#*+8rA<5#?e z*3ZlGz3?Y?H?cRxlxV^AswD(t-Q-DFpnsK1D}#)WpwXt3!gA8R?92CaCpObFSu4vm zGc3P2f~rd#diVN>&vAp1hYzAGMqU>jBUNy$!&Eg4YVtyeVAj?Z{ub#SpcXxW{QI~K zIdcin=_8jn?5UTzSW)By6!{en@t_wG2;~oCoy`z?T?W$o1e_2sWHH+;>`now&9Ce|%2`&&+@+O3DF!3( z;JaC`QnUE@RcAkt!?^dxTO5I4JU#c-S?Q=rYG(mh(lk-K$Sx;oNJ8osR zr`^tK+m$#-%P3aJ<^qsiO>!Ag} z9@EsMr;Xqp{pIx={s^m&(eX*$ZHkXS<8GO_X$m+q6!6?I4BURXligR#PRn90;THc9 zg7tw(+m`C0`U{ghO!j{|Wo#s^tLdjljY0twHdADm5ypoC2<*H$wI}vyGC}BQU8x66 z%jX7tebg{^vph8btvUdq+c)d&WS-8~OP9A`Qi~Q<39NHDy$81xRRyvTbMf=#3iX8Z z6ufi7CM1^t^JuxVkuKpKSSm02#`3e9$s4?+M{YT)GUs6ZSxb@Mz55Cvp&0^r-G=%Wj4}1u%Gj=)soi5pi}tb3y{LLN+Y2kN zeB0saMIQc9+R<51y$!{MLYYsEwSkrA!9ao+SeFOYq zlUqwzr0h&gqAg>xA(-?qShLnvxxqtI)H^;Hn~Ebpj8k$Ci3tIhjOuoZ%0!`Mi2Q0O%`)ZPE*CS=vbWLBqABnh5iI`FKKJtK z@NcJlT_$Kx$QD8Gm%9dtB%cN3jUXFd2*Ln)&CFW--d^ZBp~wy=O$vRyDX!BO^B!9T z<)_GH4_lQW)HfV9I1ETD+VLl)UgW}z)YUMYa!!$eQv3e9-1nyt_DZEu>zarvymc+~ On>vgK7Tx3?e&c`d{oy_U diff --git a/docs/google-cloud-storage-sink.md b/docs/google-cloud-storage-sink.md index 25a8f204..043e52e4 100644 --- a/docs/google-cloud-storage-sink.md +++ b/docs/google-cloud-storage-sink.md @@ -49,7 +49,7 @@ pulsarctl sinks create \ "provider": "google-cloud-storage", "bucket": "Your bucket name", "formatType": "json", - "partitioner": "topic" + "partitionerType": "PARTITION" }' ``` @@ -113,33 +113,35 @@ You can see the object at public/default/{{Your topic name}}-partition-0/xxxx.js Before using the Google Cloud Storage sink connector, you need to configure it. This table outlines the properties and the descriptions. -| Name | Type | Required | Sensitive | Default | Description | -|---------------------------------|---------|----------|-----------|--------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| `provider` | String | True | false | null | The Cloud Storage type, google cloud storage only supports the `google-cloud-storage` provider. | -| `bucket` | String | True | false | null | The Cloud Storage bucket. | -| `formatType` | String | True | false | "json" | The data format type. Available options are `json`, `avro`, `bytes`, or `parquet`. By default, it is set to `json`. | -| `partitioner` | String | False | false | null | The partitioner for partitioning the resulting files. Available options are `topic`, `time` or `legacy`. By default, it's set to `legacy`. Please see [Partitioner](#partitioner) for more details. | -| `partitionerType` | String | False | false | null | The legacy partitioning type. It can be configured by topic partitions or by time. By default, the partition type is configured by topic partitions. It only works when the partitioner is set to `legacy`. || `gcsServiceAccountKeyFileContent` | String | False | true | "" | The contents of the JSON service key file. If empty, credentials are read from `gcsServiceAccountKeyFilePath` file. | -| `gcsServiceAccountKeyFilePath` | String | False | true | "" | Path to the GCS credentials file. If empty, the credentials file will be read from the `GOOGLE_APPLICATION_CREDENTIALS` environment variable. | -| `timePartitionPattern` | String | False | false | "yyyy-MM-dd" | The format pattern of the time-based partitioning. For details, refer to the Java date and time format. | -| `timePartitionDuration` | String | False | false | "86400000" | The time interval for time-based partitioning. Support formatted interval string, such as `30d`, `24h`, `30m`, `10s`, and also support number in milliseconds precision, such as `86400000` refers to `24h` or `1d`. | -| `partitionerUseIndexAsOffset` | Boolean | False | false | false | Whether to use the Pulsar's message index as offset or the record sequence. It's recommended if the incoming messages may be batched. The brokers may or not expose the index metadata and, if it's not present on the record, the sequence will be used. See [PIP-70](https://github.com/apache/pulsar/wiki/PIP-70%3A-Introduce-lightweight-broker-entry-metadata) for more details. | -| `batchSize` | int | False | false | 10 | The number of records submitted in batch. | -| `batchTimeMs` | long | False | false | 1000 | The interval for batch submission. | -| `maxBatchBytes` | long | False | false | 10000000 | The maximum number of bytes in a batch. | -| `sliceTopicPartitionPath` | Boolean | False | false | false | When it is set to `true`, split the partitioned topic name into separate folders in the bucket path. | -| `withMetadata` | Boolean | False | false | false | Save message attributes to metadata. | -| `useHumanReadableMessageId` | Boolean | False | false | false | Use a human-readable format string for messageId in message metadata. The messageId is in a format like `ledgerId:entryId:partitionIndex:batchIndex`. Otherwise, the messageId is a Hex-encoded string. | -| `withTopicPartitionNumber` | Boolean | False | false | true | When it is set to `true`, include the topic partition number to the object path. | -| `bytesFormatTypeSeparator` | String | False | false | "0x10" | It is inserted between records for the `formatType` of bytes. By default, it is set to '0x10'. An input record that contains the line separator looks like multiple records in the output object. | -| `pendingQueueSize` | int | False | false | 10 | The number of records buffered in queue. By default, it is equal to `batchSize`. You can set it manually. | -| `useHumanReadableSchemaVersion` | Boolean | False | false | false | Use a human-readable format string for the schema version in the message metadata. If it is set to `true`, the schema version is in plain string format. Otherwise, the schema version is in hex-encoded string format. | -| `skipFailedMessages` | Boolean | False | false | false | Configure whether to skip a message which it fails to be processed. If it is set to `true`, the connector will skip the failed messages by `ack` it. Otherwise, the connector will `fail` the message. | -| `pathPrefix` | String | False | false | false | If it is set, the output files are stored in a folder under the given bucket path. The `pathPrefix` must be in the format of `xx/xxx/`. | -| `avroCodec` | String | False | false | snappy | Compression codec used when formatType=`avro`. Available compression types are: none (no compression), deflate, bzip2, xz, zstandard, snappy. | -| `parquetCodec` | String | False | false | gzip | Compression codec used when formatType=`parquet`. Available compression types are: none (no compression), snappy, gzip, lzo, brotli, lz4, zstd. | -| `jsonAllowNaN` | Boolean | False | false | false | Recognize 'NaN', 'INF', '-INF' as legal floating number values when formatType=`json`. Since JSON specification does not allow such values this is a non-standard feature and disabled by default. | -| `includeTopicToMetadata` | Boolean | False | false | false | Include the topic name to the metadata. | +| Name | Type | Required | Sensitive | Default | Description | +|-----------------------------------|---------|----------|-----------|--------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `provider` | String | True | false | null | The Cloud Storage type, google cloud storage only supports the `google-cloud-storage` provider. | +| `bucket` | String | True | false | null | The Cloud Storage bucket. | +| `formatType` | String | True | false | "json" | The data format type. Available options are `json`, `avro`, `bytes`, or `parquet`. By default, it is set to `json`. | +| `partitioner` | String | False | false | null | The partitioner for partitioning the resulting files. Available options are `topic`, `time` or `legacy`. By default, it's set to `legacy`. Please see [Partitioner](#partitioner) for more details. | +| `partitionerType` | String | False | false | null | The legacy partitioning type. It can be configured by topic partitions or by time. By default, the partition type is configured by topic partitions. It only works when the partitioner is set to `legacy`. | +| `gcsServiceAccountKeyFileContent` | String | False | true | "" | The contents of the JSON service key file. If empty, credentials are read from `gcsServiceAccountKeyFilePath` file. | +| `gcsServiceAccountKeyFilePath` | String | False | true | "" | Path to the GCS credentials file. If empty, the credentials file will be read from the `GOOGLE_APPLICATION_CREDENTIALS` environment variable. | +| `timePartitionPattern` | String | False | false | "yyyy-MM-dd" | The format pattern of the time-based partitioning. For details, refer to the Java date and time format. | +| `timePartitionDuration` | String | False | false | "86400000" | The time interval for time-based partitioning. Support formatted interval string, such as `30d`, `24h`, `30m`, `10s`, and also support number in milliseconds precision, such as `86400000` refers to `24h` or `1d`. | +| `partitionerUseIndexAsOffset` | Boolean | False | false | false | Whether to use the Pulsar's message index as offset or the record sequence. It's recommended if the incoming messages may be batched. The brokers may or not expose the index metadata and, if it's not present on the record, the sequence will be used. See [PIP-70](https://github.com/apache/pulsar/wiki/PIP-70%3A-Introduce-lightweight-broker-entry-metadata) for more details. | +| `batchSize` | int | False | false | 10 | The number of records submitted in batch. | +| `batchTimeMs` | long | False | false | 1000 | The interval for batch submission. | +| `maxBatchBytes` | long | False | false | 10000000 | The maximum number of bytes in a batch. | +| `batchModel` | Enum | False | false | BLEND | Determines how records are batched. Options: `BLEND`, `PARTITIONED`. The BLEND which combines all topic records into a single batch, optimizing for throughput, and PARTITIONED which batches records separately for each topic, maintaining topic-level separation. Note: When set to PARTITIONED, the connector will cache data up to the size of the number of subscribed topics multiplied by maxBatchBytes. This means you need to anticipate the connector's memory requirements in advance. | +| `pendingQueueSize` | int | False | false | 10 | The number of records buffered in queue. By default, it is equal to `batchSize`. You can set it manually. | +| `withMetadata` | Boolean | False | false | false | Save message attributes to metadata. | +| `includeTopicToMetadata` | Boolean | False | false | false | Include the topic name to the metadata. | +| `sliceTopicPartitionPath` | Boolean | False | false | false | When it is set to `true`, split the partitioned topic name into separate folders in the bucket path. | +| `useHumanReadableMessageId` | Boolean | False | false | false | Use a human-readable format string for messageId in message metadata. The messageId is in a format like `ledgerId:entryId:partitionIndex:batchIndex`. Otherwise, the messageId is a Hex-encoded string. | +| `withTopicPartitionNumber` | Boolean | False | false | true | When it is set to `true`, include the topic partition number to the object path. | +| `bytesFormatTypeSeparator` | String | False | false | "0x10" | It is inserted between records for the `formatType` of bytes. By default, it is set to '0x10'. An input record that contains the line separator looks like multiple records in the output object. | +| `useHumanReadableSchemaVersion` | Boolean | False | false | false | Use a human-readable format string for the schema version in the message metadata. If it is set to `true`, the schema version is in plain string format. Otherwise, the schema version is in hex-encoded string format. | +| `skipFailedMessages` | Boolean | False | false | false | Configure whether to skip a message which it fails to be processed. If it is set to `true`, the connector will skip the failed messages by `ack` it. Otherwise, the connector will `fail` the message. | +| `pathPrefix` | String | False | false | false | If it is set, the output files are stored in a folder under the given bucket path. The `pathPrefix` must be in the format of `xx/xxx/`. | +| `avroCodec` | String | False | false | snappy | Compression codec used when formatType=`avro`. Available compression types are: none (no compression), deflate, bzip2, xz, zstandard, snappy. | +| `parquetCodec` | String | False | false | gzip | Compression codec used when formatType=`parquet`. Available compression types are: none (no compression), snappy, gzip, lzo, brotli, lz4, zstd. | +| `jsonAllowNaN` | Boolean | False | false | false | Recognize 'NaN', 'INF', '-INF' as legal floating number values when formatType=`json`. Since JSON specification does not allow such values this is a non-standard feature and disabled by default. | ## Advanced features @@ -213,29 +215,16 @@ The reason for this is that the sink will only acknowledge messages after they a If this limit is lower or close to `batchSize`, the sink never receives enough messages to trigger a flush based on the amount of messages. In this case please ensure the `maxUnackedMessagesPerConsumer` set in the broker configuration is sufficiently larger than the `batchSize` setting of the sink. -### Partitioner +### Partitioner Type -The partitioner is used for partitioning the data into different files in the cloud storage. -There are three types of partitioner: +There are two types of partitioner: -- **Topic Partitioner**: Messages are partitioned according to the pre-existing partitions in the Pulsar topics. For - instance, a message for the topic `public/default/my-topic-partition-0` would be directed to the - file `public/default/my-topic-partition-0/xxx.json`, where `xxx` signifies the earliest message offset in this file. -- **Time Partitioner**: Messages are partitioned based on the timestamp at the time of flushing. For the aforementioned - message, it would be directed to the file `1703037311.json`, where `1703037311` represents the flush timestamp of the - first message in this file. -- **Legacy Partitioner**: This type reverts to the old partitioner behavior. The legacy configuration `partitionerType` would be respected. - -#### Legacy Partitioner - -There are two types of legacy partitioner: - -- **Simple partitioner**: This is the default partitioning method based on Pulsar partitions. In other words, data is +- **PARTITION**: This is the default partitioning method based on Pulsar partitions. In other words, data is partitioned according to the pre-existing partitions in Pulsar topics. For instance, a message for the topic `public/default/my-topic-partition-0` would be directed to the file `public/default/my-topic-partition-0/xxx.json`, where `xxx` signifies the earliest message offset in this file. -- **Time partitioner**: Data is partitioned according to the time it was flushed. Using the previous message as an +- **TIME**: Data is partitioned according to the time it was flushed. Using the previous message as an example, if it was received on 2023-12-20, it would be directed to `public/default/my-topic-partition-0/2023-12-20/xxx.json`, where `xxx` also denotes the earliest message offset in this file. \ No newline at end of file diff --git a/src/main/java/org/apache/pulsar/io/jcloud/BlobStoreAbstractConfig.java b/src/main/java/org/apache/pulsar/io/jcloud/BlobStoreAbstractConfig.java index e1ce1518..4b4f4a4f 100644 --- a/src/main/java/org/apache/pulsar/io/jcloud/BlobStoreAbstractConfig.java +++ b/src/main/java/org/apache/pulsar/io/jcloud/BlobStoreAbstractConfig.java @@ -36,13 +36,13 @@ import lombok.experimental.Accessors; import org.apache.commons.lang3.EnumUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.io.jcloud.batch.BatchModel; import org.apache.pulsar.io.jcloud.format.AvroFormat; import org.apache.pulsar.io.jcloud.format.BytesFormat; import org.apache.pulsar.io.jcloud.format.Format; import org.apache.pulsar.io.jcloud.format.JsonFormat; import org.apache.pulsar.io.jcloud.format.ParquetFormat; import org.apache.pulsar.io.jcloud.partitioner.PartitionerType; -import org.apache.pulsar.io.jcloud.partitioner.legacy.LegacyPartitionerType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -70,55 +70,48 @@ public class BlobStoreAbstractConfig implements Serializable { public static final String PROVIDER_GCS = "google-cloud-storage"; public static final String PROVIDER_AZURE = "azure-blob-storage"; + // #### bucket configuration #### private String provider; - private String bucket; - private String region; - private String endpoint; - private String pathPrefix; - - private String formatType; - - @Deprecated // Use partitioner instead - private String partitionerType = "partition"; - private PartitionerType partitioner = PartitionerType.LEGACY; + // #### common configuration #### + private boolean usePathStyleUrl = true; + private String awsCannedAcl = ""; + private boolean skipFailedMessages = false; + // #### partitioner configuration #### + // Options: PARTITION, TIME + private String partitionerType; + private String pathPrefix; + private boolean withTopicPartitionNumber = true; private boolean partitionerUseIndexAsOffset; - - // The AVRO codec. - // Options: none, deflate, bzip2, xz, zstandard, snappy - private String avroCodec = "snappy"; - - // The Parquet codec. - // Options: none, snappy, gzip, lzo, brotli, lz4, zstd - private String parquetCodec = "gzip"; - private String timePartitionPattern; - private String timePartitionDuration; - private boolean sliceTopicPartitionPath; + // #### format configuration #### + private String formatType; + // The AVRO codec: none, deflate, bzip2, xz, zstandard, snappy + private String avroCodec = "snappy"; + // The Parquet codec: none, snappy, gzip, lzo, brotli, lz4, zstd + private String parquetCodec = "gzip"; + private String bytesFormatTypeSeparator = "0x10"; + private boolean jsonAllowNaN = false; + + // #### batch configuration #### private long maxBatchBytes = 10_000_000; private int batchSize = 10; - private int pendingQueueSize = -1; - private long batchTimeMs = 1000; + private BatchModel batchModel = BatchModel.BLEND; + private int pendingQueueSize = -1; - private boolean usePathStyleUrl = true; - private String awsCannedAcl = ""; - + // #### metadata configuration #### private boolean withMetadata; private boolean useHumanReadableMessageId; private boolean useHumanReadableSchemaVersion; private boolean includeTopicToMetadata; - private boolean withTopicPartitionNumber = true; - private String bytesFormatTypeSeparator = "0x10"; - private boolean skipFailedMessages = false; - private boolean jsonAllowNaN = false; public void validate() { checkNotNull(provider, "provider not set."); @@ -136,18 +129,18 @@ public void validate() { } if (partitionerType == null - || (EnumUtils.getEnumIgnoreCase(LegacyPartitionerType.class, partitionerType) == null + || (EnumUtils.getEnumIgnoreCase(PartitionerType.class, partitionerType) == null && !partitionerType.equalsIgnoreCase("default"))) { // `default` option is for backward compatibility throw new IllegalArgumentException( "partitionerType property not set properly, available options: " - + Arrays.stream(LegacyPartitionerType.values()) + + Arrays.stream(PartitionerType.values()) .map(Enum::name) .map(String::toLowerCase) .collect(Collectors.joining(",")) ); } - if (LegacyPartitionerType.TIME.name().equalsIgnoreCase(partitionerType)) { + if (PartitionerType.TIME.name().equalsIgnoreCase(partitionerType)) { if (StringUtils.isNoneBlank(timePartitionPattern)) { LOGGER.info("test timePartitionPattern is ok {} {}", timePartitionPattern, @@ -168,6 +161,7 @@ public void validate() { checkArgument(StringUtils.endsWith(pathPrefix, "/"), "pathPrefix must end with '/',the style is 'xx/xxx/'."); } + pathPrefix = StringUtils.trimToEmpty(pathPrefix); if ("bytes".equalsIgnoreCase(formatType)) { checkArgument(StringUtils.isNotEmpty(bytesFormatTypeSeparator), diff --git a/src/main/java/org/apache/pulsar/io/jcloud/batch/BatchContainer.java b/src/main/java/org/apache/pulsar/io/jcloud/batch/BatchContainer.java new file mode 100644 index 00000000..9f091d06 --- /dev/null +++ b/src/main/java/org/apache/pulsar/io/jcloud/batch/BatchContainer.java @@ -0,0 +1,103 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.io.jcloud.batch; + +import com.google.common.collect.Lists; +import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.functions.api.Record; + +/** + * BatchContainer is used to store and manage batches of records. + * It keeps track of the current batch size and bytes, and checks + * if the batch needs to be flushed based on the max batch size, bytes, and time. + */ +public class BatchContainer { + + private final long maxBatchSize; + private final long maxBatchBytes; + private final long maxBatchTimeMs; + private final AtomicLong currentBatchSize = new AtomicLong(0L); + private final AtomicLong currentBatchBytes = new AtomicLong(0L); + private final ArrayBlockingQueue> pendingFlushQueue; + private volatile long lastPoolRecordsTime; + + public BatchContainer(long maxBatchSize, long maxBatchBytes, long maxBatchTimeMs, int maxPendingQueueSize) { + this.maxBatchSize = maxBatchSize; + this.maxBatchBytes = maxBatchBytes; + this.maxBatchTimeMs = maxBatchTimeMs; + this.pendingFlushQueue = new ArrayBlockingQueue<>(maxPendingQueueSize); + this.lastPoolRecordsTime = System.currentTimeMillis(); + } + + public void add(Record record) throws InterruptedException { + pendingFlushQueue.put(record); + updateCurrentBatchSize(1); + updateCurrentBatchBytes(record.getMessage().get().size()); + } + + public long getCurrentBatchSize() { + return currentBatchSize.get(); + } + + public long getCurrentBatchBytes() { + return currentBatchBytes.get(); + } + + public void updateCurrentBatchSize(long delta) { + currentBatchSize.addAndGet(delta); + } + + public void updateCurrentBatchBytes(long delta) { + currentBatchBytes.addAndGet(delta); + } + + public boolean isEmpty() { + return pendingFlushQueue.isEmpty(); + } + + public boolean needFlush() { + long currentTime = System.currentTimeMillis(); + return currentBatchSize.get() >= maxBatchSize + || currentBatchBytes.get() >= maxBatchBytes + || (currentTime - lastPoolRecordsTime) >= maxBatchTimeMs; + } + + public List> pollNeedFlushRecords() { + final List> needFlushRecords = Lists.newArrayList(); + long recordsToInsertBytes = 0; + while (!pendingFlushQueue.isEmpty() && needFlushRecords.size() < maxBatchSize + && recordsToInsertBytes < maxBatchBytes) { + Record r = pendingFlushQueue.poll(); + if (r != null) { + if (r.getMessage().isPresent()) { + long recordBytes = r.getMessage().get().size(); + recordsToInsertBytes += recordBytes; + } + needFlushRecords.add(r); + } + } + updateCurrentBatchBytes(-1 * recordsToInsertBytes); + updateCurrentBatchSize(-1 * needFlushRecords.size()); + lastPoolRecordsTime = System.currentTimeMillis(); + return needFlushRecords; + } +} diff --git a/src/main/java/org/apache/pulsar/io/jcloud/batch/BatchManager.java b/src/main/java/org/apache/pulsar/io/jcloud/batch/BatchManager.java new file mode 100644 index 00000000..6499c81b --- /dev/null +++ b/src/main/java/org/apache/pulsar/io/jcloud/batch/BatchManager.java @@ -0,0 +1,148 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.io.jcloud.batch; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.functions.api.Record; +import org.apache.pulsar.io.jcloud.BlobStoreAbstractConfig; +import reactor.util.function.Tuple2; + +/** + * The BatchManager interface defines the operations that a batch manager should provide. + * A batch manager handles the batching of records for more efficient processing. + */ +public interface BatchManager { + + /** + * Creates a BatchManager based on the provided configuration. + * @param config the configuration to use when creating the BatchManager + * @return a BatchManager instance + * @throws IllegalArgumentException if the batch model specified in the configuration is unsupported + */ + static BatchManager createBatchManager(BlobStoreAbstractConfig config) { + switch (config.getBatchModel()) { + case BLEND: + return new BlendBatchManager(config.getBatchSize(), config.getMaxBatchBytes(), + config.getBatchTimeMs(), config.getPendingQueueSize()); + case PARTITIONED: + return new PartitionedBatchManager(config.getBatchSize(), config.getMaxBatchBytes(), + config.getBatchTimeMs(), config.getPendingQueueSize()); + default: + throw new IllegalArgumentException("Unsupported batch model: " + config.getBatchModel()); + } + } + + /** + * Calculate the sum of the byte sizes of the messages in a list of records. + * + * @param records The list of records whose message sizes are to be summed. + * @return The sum of the byte sizes of the messages in the given records. + */ + static long getBytesSum(List> records) { + return records.stream() + .map(Record::getMessage) + .filter(Optional::isPresent) + .map(Optional::get) + .mapToLong(Message::size) + .sum(); + } + + /** + * Adds a record to the batch manager. + * @param record the record to add + * @throws InterruptedException if the adding process is interrupted + */ + void add(Record record) throws InterruptedException; + + /** + * Determines whether the current batch needs to be flushed. + * @return true if the batch needs to be flushed, false otherwise + */ + boolean needFlush(); + + /** + * Retrieves the data that needs to be flushed. + * @return a map where the keys are the topic names and the values are the lists of records for each topic + */ + Map>> pollNeedFlushData(); + + /** + * Retrieves the current batch size for a given topic. + * @param topicName the name of the topic + * @return the current batch size + */ + long getCurrentBatchSize(String topicName); + + /** + * Retrieves the current batch bytes for a given topic. + * @param topicName the name of the topic + * @return the current batch bytes + */ + long getCurrentBatchBytes(String topicName); + + /** + * Retrieves the current batch statistics for all topics. + * @return Map> + */ + Map> getCurrentStats(); + + /** + * Retrieves the current batch statistics for all topics as a string. + * @return a string representation of the current batch statistics + */ + default String getCurrentStatsStr() { + Map> stats = getCurrentStats(); + StringBuilder sb = new StringBuilder(); + sb.append("{"); + stats.forEach((topic, tuple) -> { + sb.append(String.format("\"%s\": {\"currentBatchSize\": %d, \"currentBatchBytes\": %d}, ", + topic, tuple.getT1(), tuple.getT2())); + }); + if (!stats.isEmpty()) { + sb.setLength(sb.length() - 2); + } + sb.append("}"); + return sb.toString(); + } + + + /** + * Updates the current batch size for a given topic. + * @param topicName the name of the topic + * @param delta the amount to add to the current batch size + */ + void updateCurrentBatchSize(String topicName, long delta); + + /** + * Updates the current batch bytes for a given topic. + * @param topicName the name of the topic + * @param delta the amount to add to the current batch bytes + */ + void updateCurrentBatchBytes(String topicName, long delta); + + /** + * Determines whether the batch manager is currently empty. + * @return true if the batch manager is empty, false otherwise + */ + boolean isEmpty(); +} diff --git a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/LegacyPartitionerType.java b/src/main/java/org/apache/pulsar/io/jcloud/batch/BatchModel.java similarity index 51% rename from src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/LegacyPartitionerType.java rename to src/main/java/org/apache/pulsar/io/jcloud/batch/BatchModel.java index 83960a7e..d8129665 100644 --- a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/LegacyPartitionerType.java +++ b/src/main/java/org/apache/pulsar/io/jcloud/batch/BatchModel.java @@ -16,12 +16,23 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.io.jcloud.partitioner.legacy; +package org.apache.pulsar.io.jcloud.batch; /** - * partitioner types. + * Enum representing the different batch models. */ -public enum LegacyPartitionerType { - PARTITION, - TIME; +public enum BatchModel { + /** + * BlendBatchManager is a type of BatchManager that uses a single BatchContainer + * for all topics. This means that all records, regardless of topic, are batched together. + */ + BLEND, + /** + * PartitionedBatchManager is a type of BatchManager that uses separate BatchContainers + * for each topic. This means that records are batched separately for each topic. + * Note: When set to PARTITIONED, the connector will cache data up to the size of the + * number of subscribed topics multiplied by maxBatchBytes. This means you need to anticipate the connector + * memory requirements in advance. + */ + PARTITIONED } diff --git a/src/main/java/org/apache/pulsar/io/jcloud/batch/BlendBatchManager.java b/src/main/java/org/apache/pulsar/io/jcloud/batch/BlendBatchManager.java new file mode 100644 index 00000000..bf74ffb7 --- /dev/null +++ b/src/main/java/org/apache/pulsar/io/jcloud/batch/BlendBatchManager.java @@ -0,0 +1,90 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.io.jcloud.batch; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.functions.api.Record; +import reactor.util.function.Tuple2; +import reactor.util.function.Tuples; + +/** + * BlendBatchManager is a type of BatchManager that uses a single BatchContainer + * for all topics. This means that all records, regardless of topic, are batched together. + */ +@Slf4j +public class BlendBatchManager implements BatchManager { + + private final BatchContainer batchContainer; + + public BlendBatchManager(long maxBatchSize, long maxBatchBytes, long maxBatchTimeMs, int maxPendingQueueSize) { + batchContainer = new BatchContainer(maxBatchSize, maxBatchBytes, maxBatchTimeMs, maxPendingQueueSize); + } + + @Override + public void add(Record record) throws InterruptedException { + batchContainer.add(record); + } + + @Override + public long getCurrentBatchSize(String topicName) { + return batchContainer.getCurrentBatchSize(); + } + + @Override + public long getCurrentBatchBytes(String topicName) { + return batchContainer.getCurrentBatchBytes(); + } + + @Override + public Map> getCurrentStats() { + return Map.of("ALL", Tuples.of(batchContainer.getCurrentBatchSize(), batchContainer.getCurrentBatchBytes())); + } + + @Override + public void updateCurrentBatchSize(String topicName, long delta) { + batchContainer.updateCurrentBatchSize(delta); + } + + @Override + public void updateCurrentBatchBytes(String topicName, long delta) { + batchContainer.updateCurrentBatchBytes(delta); + } + + @Override + public boolean isEmpty() { + return batchContainer.isEmpty(); + } + + @Override + public boolean needFlush() { + return batchContainer.needFlush(); + } + + public Map>> pollNeedFlushData() { + if (!needFlush()) { + return Map.of(); + } + List> records = batchContainer.pollNeedFlushRecords(); + return records.stream().collect(Collectors.groupingBy(record -> record.getTopicName().get())); + } +} diff --git a/src/main/java/org/apache/pulsar/io/jcloud/batch/PartitionedBatchManager.java b/src/main/java/org/apache/pulsar/io/jcloud/batch/PartitionedBatchManager.java new file mode 100644 index 00000000..dd167e30 --- /dev/null +++ b/src/main/java/org/apache/pulsar/io/jcloud/batch/PartitionedBatchManager.java @@ -0,0 +1,109 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.io.jcloud.batch; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.functions.api.Record; +import reactor.util.function.Tuple2; +import reactor.util.function.Tuples; + +/** + * PartitionedBatchManager is a type of BatchManager that uses separate BatchContainers + * for each topic. This means that records are batched separately for each topic. + */ +public class PartitionedBatchManager implements BatchManager { + + private final long maxBatchSize; + private final long maxBatchBytes; + private final long maxBatchTimeMs; + private final int maxPendingQueueSize; + private final Map topicBatchContainer; + + public PartitionedBatchManager(long maxBatchSize, long maxBatchBytes, + long maxBatchTimeMs, int maxPendingQueueSize) { + this.maxBatchSize = maxBatchSize; + this.maxBatchBytes = maxBatchBytes; + this.maxBatchTimeMs = maxBatchTimeMs; + this.maxPendingQueueSize = maxPendingQueueSize; + this.topicBatchContainer = new ConcurrentHashMap<>(); + } + + public void add(Record record) throws InterruptedException { + String topicName = record.getTopicName() + .orElseThrow(() -> new IllegalArgumentException("Topic name cannot be null")); + getBatchContainer(topicName).add(record); + } + + public long getCurrentBatchSize(String topicName) { + return getBatchContainer(topicName).getCurrentBatchSize(); + } + + public long getCurrentBatchBytes(String topicName) { + return getBatchContainer(topicName).getCurrentBatchBytes(); + } + + @Override + public Map> getCurrentStats() { + Map> stats = new HashMap<>(); + topicBatchContainer.forEach((topicName, batchContainer) -> { + long currentBatchSize = batchContainer.getCurrentBatchSize(); + long currentBatchBytes = batchContainer.getCurrentBatchBytes(); + stats.put(topicName, Tuples.of(currentBatchSize, currentBatchBytes)); + }); + return stats; + } + + public void updateCurrentBatchSize(String topicName, long delta) { + getBatchContainer(topicName).updateCurrentBatchSize(delta); + } + + public void updateCurrentBatchBytes(String topicName, long delta) { + getBatchContainer(topicName).updateCurrentBatchBytes(delta); + } + + public boolean isEmpty() { + return topicBatchContainer.values().stream().allMatch(BatchContainer::isEmpty); + } + + public boolean needFlush() { + return topicBatchContainer.values().stream().anyMatch(BatchContainer::needFlush); + } + + public Map>> pollNeedFlushData() { + Map>> flushData = new HashMap<>(); + topicBatchContainer.forEach((topicName, batchContainer) -> { + if (batchContainer.needFlush()) { + List> records = batchContainer.pollNeedFlushRecords(); + if (!records.isEmpty()) { + flushData.put(topicName, records); + } + } + }); + return flushData; + } + + private BatchContainer getBatchContainer(String topicName) { + return topicBatchContainer.computeIfAbsent(topicName, + k -> new BatchContainer(maxBatchSize, maxBatchBytes, maxBatchTimeMs, maxPendingQueueSize)); + } +} diff --git a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/package-info.java b/src/main/java/org/apache/pulsar/io/jcloud/batch/package-info.java similarity index 93% rename from src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/package-info.java rename to src/main/java/org/apache/pulsar/io/jcloud/batch/package-info.java index 1b8a8842..f5beef23 100644 --- a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/package-info.java +++ b/src/main/java/org/apache/pulsar/io/jcloud/batch/package-info.java @@ -16,4 +16,4 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.io.jcloud.partitioner.legacy; +package org.apache.pulsar.io.jcloud.batch; \ No newline at end of file diff --git a/src/main/java/org/apache/pulsar/io/jcloud/format/Format.java b/src/main/java/org/apache/pulsar/io/jcloud/format/Format.java index f8661f56..2d8f727a 100644 --- a/src/main/java/org/apache/pulsar/io/jcloud/format/Format.java +++ b/src/main/java/org/apache/pulsar/io/jcloud/format/Format.java @@ -20,15 +20,44 @@ import java.nio.ByteBuffer; import java.util.Iterator; +import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.schema.GenericRecord; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.functions.api.Record; +import org.apache.pulsar.io.jcloud.BlobStoreAbstractConfig; import org.apache.pulsar.jcloud.shade.com.google.common.io.ByteSource; /** * output record format. */ public interface Format { + + static Format buildFormat(BlobStoreAbstractConfig sinkConfig) { + String formatType = StringUtils.defaultIfBlank(sinkConfig.getFormatType(), "json"); + Format format; + switch (formatType) { + case "avro": + format = new AvroFormat(); + break; + case "parquet": + format = new ParquetFormat(); + break; + case "json": + format = new JsonFormat(); + break; + case "bytes": + format = new BytesFormat(); + break; + default: + throw new RuntimeException("not support formatType " + formatType); + } + InitConfiguration formatConfigInitializer = + (InitConfiguration) format; + formatConfigInitializer.configure(sinkConfig); + return format; + } + /** * get format extension. * diff --git a/src/main/java/org/apache/pulsar/io/jcloud/format/ParquetFormat.java b/src/main/java/org/apache/pulsar/io/jcloud/format/ParquetFormat.java index 93efd12e..0b8ba94f 100644 --- a/src/main/java/org/apache/pulsar/io/jcloud/format/ParquetFormat.java +++ b/src/main/java/org/apache/pulsar/io/jcloud/format/ParquetFormat.java @@ -45,10 +45,10 @@ import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.io.jcloud.BlobStoreAbstractConfig; -import org.apache.pulsar.io.jcloud.BytesOutputStream; import org.apache.pulsar.io.jcloud.format.parquet.ProtobufParquetWriter; import org.apache.pulsar.io.jcloud.format.parquet.proto.Metadata; import org.apache.pulsar.io.jcloud.util.AvroRecordUtil; +import org.apache.pulsar.io.jcloud.util.BytesOutputStream; import org.apache.pulsar.io.jcloud.util.MetadataUtil; /** diff --git a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/AbstractPartitioner.java b/src/main/java/org/apache/pulsar/io/jcloud/partitioner/AbstractPartitioner.java similarity index 98% rename from src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/AbstractPartitioner.java rename to src/main/java/org/apache/pulsar/io/jcloud/partitioner/AbstractPartitioner.java index 71e2d3af..0b95f309 100644 --- a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/AbstractPartitioner.java +++ b/src/main/java/org/apache/pulsar/io/jcloud/partitioner/AbstractPartitioner.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.io.jcloud.partitioner.legacy; +package org.apache.pulsar.io.jcloud.partitioner; import java.util.ArrayList; import java.util.List; diff --git a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/LegacyPartitioner.java b/src/main/java/org/apache/pulsar/io/jcloud/partitioner/LegacyPartitioner.java deleted file mode 100644 index 38dab9b6..00000000 --- a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/LegacyPartitioner.java +++ /dev/null @@ -1,61 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.io.jcloud.partitioner; - -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.client.api.schema.GenericRecord; -import org.apache.pulsar.functions.api.Record; -import org.apache.pulsar.io.jcloud.format.Format; - -/** - * The LegacyPartitioner class, implementing the Partitioner interface, is designed to partition records according to - * their respective topic names. - * It is used to partition records based on their topic name. It will then use the legacy partitioner such as - * SimplePartitioner or TimePartitioner as the underlying partitioner. This is for the backward compatibility usage. - */ -@Slf4j -public class LegacyPartitioner implements Partitioner { - - /** - * This method partitions a list of records into a map where the keys are the topic names and the values are lists - * of records. - * - * @param records A list of records of type GenericRecord that need to be partitioned. - * @return A map where the keys are the topic names and the values are lists of records. - */ - @Override - public Map>> partition(List> records) { - return records.stream().collect(Collectors.groupingBy(record -> record.getTopicName().get())); - } - - public String buildPartitionPath(Record message, String pathPrefix, - org.apache.pulsar.io.jcloud.partitioner.legacy.Partitioner p, - Format format, - long partitioningTimestamp) { - - String encodePartition = p.encodePartition(message, partitioningTimestamp); - String partitionedPath = p.generatePartitionedPath(message.getTopicName().get(), encodePartition); - String path = pathPrefix + partitionedPath + format.getExtension(); - log.info("generate message[recordSequence={}] savePath: {}", message.getRecordSequence().get(), path); - return path; - } -} diff --git a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/Partitioner.java b/src/main/java/org/apache/pulsar/io/jcloud/partitioner/Partitioner.java index 9700d074..b426cabb 100644 --- a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/Partitioner.java +++ b/src/main/java/org/apache/pulsar/io/jcloud/partitioner/Partitioner.java @@ -18,23 +18,69 @@ */ package org.apache.pulsar.io.jcloud.partitioner; -import java.util.List; -import java.util.Map; +import java.io.File; +import org.apache.commons.lang3.EnumUtils; import org.apache.pulsar.client.api.schema.GenericRecord; import org.apache.pulsar.functions.api.Record; +import org.apache.pulsar.io.jcloud.BlobStoreAbstractConfig; /** - * The Partitioner interface offers a mechanism to categorize a list of records into distinct parts. + * Partition incoming records, and generates directories and file names in which to store the + * incoming records. + * + * @param The type representing the field schemas. */ -public interface Partitioner { +public interface Partitioner { + + String PATH_SEPARATOR = File.separator; + + static Partitioner buildPartitioner(BlobStoreAbstractConfig sinkConfig) { + Partitioner partitioner; + String partitionerTypeName = sinkConfig.getPartitionerType(); + PartitionerType partitionerType = + EnumUtils.getEnumIgnoreCase(PartitionerType.class, partitionerTypeName, PartitionerType.PARTITION); + switch (partitionerType) { + case TIME: + partitioner = new TimePartitioner<>(); + break; + case PARTITION: + default: + partitioner = new SimplePartitioner<>(); + break; + } + partitioner.configure(sinkConfig); + return partitioner; + } + + + void configure(BlobStoreAbstractConfig config); + + /** + * Returns string representing the output path for a sinkRecord to be encoded and stored. + * + * @param sinkRecord The record to be stored by the Sink Connector + * @return The path/filename the SinkRecord will be stored into after it is encoded + */ + String encodePartition(Record sinkRecord); + + /** + * Returns string representing the output path for a sinkRecord to be encoded and stored. + * + * @param sinkRecord The record to be stored by the Sink Connector + * @param nowInMillis The current time in ms. Some Partitioners will use this option, but by + * default it is unused. + * @return The path/filename the SinkRecord will be stored into after it is encoded + */ + default String encodePartition(Record sinkRecord, long nowInMillis) { + return encodePartition(sinkRecord); + } + /** - * The partition method takes a list of records and returns a map. Each key in the map represents a - * unique partition, and the corresponding value is a list of records that belong to that partition. + * Generate saved path. * - * @param records A list of records to be partitioned. Each record is of the type GenericRecord. - * @return A map where keys represent unique partitions and values are lists of records - * associated with their respective partitions. The unique partition is consistently used as a file path in the - * cloud storage system. + * @param topic topic name + * @param encodedPartition Path encoded by the implementation class + * @return saved path */ - Map>> partition(List> records); + String generatePartitionedPath(String topic, String encodedPartition); } diff --git a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/PartitionerType.java b/src/main/java/org/apache/pulsar/io/jcloud/partitioner/PartitionerType.java index bc757dcd..addd548e 100644 --- a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/PartitionerType.java +++ b/src/main/java/org/apache/pulsar/io/jcloud/partitioner/PartitionerType.java @@ -18,20 +18,10 @@ */ package org.apache.pulsar.io.jcloud.partitioner; -import com.fasterxml.jackson.annotation.JsonCreator; - +/** + * partitioner types. + */ public enum PartitionerType { - LEGACY, - TIME, - TOPIC; - - @JsonCreator - public static PartitionerType forValue(String value) { - for (PartitionerType partitionerType : PartitionerType.values()) { - if (partitionerType.name().equalsIgnoreCase(value)) { - return partitionerType; - } - } - throw new IllegalArgumentException("Invalid partitionerType value: " + value); - } + PARTITION, + TIME; } diff --git a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/SimplePartitioner.java b/src/main/java/org/apache/pulsar/io/jcloud/partitioner/SimplePartitioner.java similarity index 95% rename from src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/SimplePartitioner.java rename to src/main/java/org/apache/pulsar/io/jcloud/partitioner/SimplePartitioner.java index fceca019..4fd9d61d 100644 --- a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/SimplePartitioner.java +++ b/src/main/java/org/apache/pulsar/io/jcloud/partitioner/SimplePartitioner.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.io.jcloud.partitioner.legacy; +package org.apache.pulsar.io.jcloud.partitioner; import org.apache.pulsar.functions.api.Record; diff --git a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/TimePartitioner.java b/src/main/java/org/apache/pulsar/io/jcloud/partitioner/TimePartitioner.java index c254684f..5803545b 100644 --- a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/TimePartitioner.java +++ b/src/main/java/org/apache/pulsar/io/jcloud/partitioner/TimePartitioner.java @@ -18,25 +18,95 @@ */ package org.apache.pulsar.io.jcloud.partitioner; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import org.apache.pulsar.client.api.schema.GenericRecord; +import java.time.Instant; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.util.function.Supplier; +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.functions.api.Record; +import org.apache.pulsar.io.jcloud.BlobStoreAbstractConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** - * The TimePartitioner is used to partition records based on the current sink timestamp. + * partition by day, hour. + * + * @param */ -public class TimePartitioner implements Partitioner { - /** - * Partitions a list of records into a map, where the key is the current system time in milliseconds - * and the value is the list of records. - * - * @param records A list of records of type GenericRecord that need to be partitioned. - * @return A map where the key is the current system time in milliseconds and the value is the list of records. - */ +public class TimePartitioner extends AbstractPartitioner { + + private static final Logger LOGGER = LoggerFactory.getLogger(TimePartitioner.class); + + private static final long DEFAULT_PARTITION_DURATION = 24 * 3600 * 1000L; + private static final String DEFAULT_PARTITION_PATTERN = "yyyy-MM-dd"; + private long partitionDuration; + private String formatString; + private DateTimeFormatter dateTimeFormatter; + + @Override + public void configure(BlobStoreAbstractConfig config) { + super.configure(config); + this.formatString = StringUtils.defaultIfBlank(config.getTimePartitionPattern(), DEFAULT_PARTITION_PATTERN); + this.partitionDuration = parseDurationString(config.getTimePartitionDuration()); + this.dateTimeFormatter = new DateTimeFormatterBuilder() + .appendPattern(formatString) + .toFormatter(); + } + + private long parseDurationString(String timePartitionDuration) { + if (StringUtils.isBlank(timePartitionDuration)) { + return DEFAULT_PARTITION_DURATION; + } + if (Character.isAlphabetic(timePartitionDuration.charAt(timePartitionDuration.length() - 1))) { + String number = timePartitionDuration.substring(0, timePartitionDuration.length() - 1); + switch (timePartitionDuration.charAt(timePartitionDuration.length() - 1)) { + case 'd': + case 'D': + return Long.parseLong(number) * 24L * 3600L * 1000L; + case 'h': + case 'H': + return Long.parseLong(number) * 3600L * 1000L; + case 'm': + return Long.parseLong(number) * 60L * 1000L; + case 's': + return Long.parseLong(number) * 1000L; + default: + throw new RuntimeException("not supported time duration scale " + timePartitionDuration); + } + } else { + try { + return Long.parseLong(timePartitionDuration); + } catch (NumberFormatException ex) { + throw new RuntimeException("not supported time duration format " + timePartitionDuration, ex); + } + } + } + @Override - public Map>> partition(List> records) { - return Collections.singletonMap(Long.toString(System.currentTimeMillis()), records); + public String encodePartition(Record sinkRecord) { + throw new RuntimeException(new IllegalAccessException()); + } + + @Override + public String encodePartition(Record sinkRecord, long nowInMillis) { + long publishTime = getPublishTime(sinkRecord, nowInMillis); + long parsed = (publishTime / partitionDuration) * partitionDuration; + String timeString = dateTimeFormatter.format(Instant.ofEpochMilli(parsed).atOffset(ZoneOffset.UTC)); + final String result = timeString + + PATH_SEPARATOR + + getMessageOffset(sinkRecord); + return result; + } + + private long getPublishTime(Record sinkRecord, Long defaultTime) { + final Supplier defaultTimeSupplier = () -> { + LOGGER.warn("record not exist Message {}", sinkRecord.getRecordSequence().get()); + return defaultTime; + }; + return sinkRecord.getMessage() + .map(Message::getPublishTime) + .orElseGet(defaultTimeSupplier); } } diff --git a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/TopicPartitioner.java b/src/main/java/org/apache/pulsar/io/jcloud/partitioner/TopicPartitioner.java deleted file mode 100644 index e337554e..00000000 --- a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/TopicPartitioner.java +++ /dev/null @@ -1,59 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.io.jcloud.partitioner; - -import java.io.File; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import org.apache.commons.lang3.StringUtils; -import org.apache.pulsar.client.api.schema.GenericRecord; -import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.functions.api.Record; - -/** - * The TopicPartitioner is used to partition records based on the topic name. - */ -public class TopicPartitioner implements Partitioner { - @Override - public Map>> partition(List> records) { - return records.stream() - .collect(Collectors.groupingBy(record -> record.getTopicName() - .orElseThrow(() -> new RuntimeException("Topic name is not present in record.")))) - .entrySet().stream() - .collect(Collectors.toMap(entry -> generateFilePath(entry.getKey(), records), Map.Entry::getValue)); - } - - String generateFilePath(String topic, List> records) { - TopicName topicName = TopicName.getPartitionedTopicName(topic); - - return StringUtils.join(Arrays.asList( - topicName.getTenant(), - topicName.getNamespacePortion(), - topicName.getLocalName(), - Long.toString(getMessageOffset(records.get(0))) - ), File.separator); - } - - protected long getMessageOffset(Record record) { - return record.getRecordSequence() - .orElseThrow(() -> new RuntimeException("The record sequence is not present in record.")); - } -} diff --git a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/Partitioner.java b/src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/Partitioner.java deleted file mode 100644 index be2f0247..00000000 --- a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/Partitioner.java +++ /dev/null @@ -1,66 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.io.jcloud.partitioner.legacy; - -import java.io.File; -import org.apache.pulsar.functions.api.Record; -import org.apache.pulsar.io.jcloud.BlobStoreAbstractConfig; - -/** - * Partition incoming records, and generates directories and file names in which to store the - * incoming records. - * - * @param The type representing the field schemas. - */ -public interface Partitioner { - - String PATH_SEPARATOR = File.separator; - - - void configure(BlobStoreAbstractConfig config); - - /** - * Returns string representing the output path for a sinkRecord to be encoded and stored. - * - * @param sinkRecord The record to be stored by the Sink Connector - * @return The path/filename the SinkRecord will be stored into after it is encoded - */ - String encodePartition(Record sinkRecord); - - /** - * Returns string representing the output path for a sinkRecord to be encoded and stored. - * - * @param sinkRecord The record to be stored by the Sink Connector - * @param nowInMillis The current time in ms. Some Partitioners will use this option, but by - * default it is unused. - * @return The path/filename the SinkRecord will be stored into after it is encoded - */ - default String encodePartition(Record sinkRecord, long nowInMillis) { - return encodePartition(sinkRecord); - } - - /** - * Generate saved path. - * - * @param topic topic name - * @param encodedPartition Path encoded by the implementation class - * @return saved path - */ - String generatePartitionedPath(String topic, String encodedPartition); -} diff --git a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/TimePartitioner.java b/src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/TimePartitioner.java deleted file mode 100644 index 30f593ed..00000000 --- a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/legacy/TimePartitioner.java +++ /dev/null @@ -1,112 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.io.jcloud.partitioner.legacy; - -import java.time.Instant; -import java.time.ZoneOffset; -import java.time.format.DateTimeFormatter; -import java.time.format.DateTimeFormatterBuilder; -import java.util.function.Supplier; -import org.apache.commons.lang3.StringUtils; -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.functions.api.Record; -import org.apache.pulsar.io.jcloud.BlobStoreAbstractConfig; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * partition by day, hour. - * - * @param - */ -public class TimePartitioner extends AbstractPartitioner { - - private static final Logger LOGGER = LoggerFactory.getLogger(TimePartitioner.class); - - private static final long DEFAULT_PARTITION_DURATION = 24 * 3600 * 1000L; - private static final String DEFAULT_PARTITION_PATTERN = "yyyy-MM-dd"; - private long partitionDuration; - private String formatString; - private DateTimeFormatter dateTimeFormatter; - - @Override - public void configure(BlobStoreAbstractConfig config) { - super.configure(config); - this.formatString = StringUtils.defaultIfBlank(config.getTimePartitionPattern(), DEFAULT_PARTITION_PATTERN); - this.partitionDuration = parseDurationString(config.getTimePartitionDuration()); - this.dateTimeFormatter = new DateTimeFormatterBuilder() - .appendPattern(formatString) - .toFormatter(); - } - - private long parseDurationString(String timePartitionDuration) { - if (StringUtils.isBlank(timePartitionDuration)) { - return DEFAULT_PARTITION_DURATION; - } - if (Character.isAlphabetic(timePartitionDuration.charAt(timePartitionDuration.length() - 1))) { - String number = timePartitionDuration.substring(0, timePartitionDuration.length() - 1); - switch (timePartitionDuration.charAt(timePartitionDuration.length() - 1)) { - case 'd': - case 'D': - return Long.parseLong(number) * 24L * 3600L * 1000L; - case 'h': - case 'H': - return Long.parseLong(number) * 3600L * 1000L; - case 'm': - return Long.parseLong(number) * 60L * 1000L; - case 's': - return Long.parseLong(number) * 1000L; - default: - throw new RuntimeException("not supported time duration scale " + timePartitionDuration); - } - } else { - try { - return Long.parseLong(timePartitionDuration); - } catch (NumberFormatException ex) { - throw new RuntimeException("not supported time duration format " + timePartitionDuration, ex); - } - } - } - - @Override - public String encodePartition(Record sinkRecord) { - throw new RuntimeException(new IllegalAccessException()); - } - - @Override - public String encodePartition(Record sinkRecord, long nowInMillis) { - long publishTime = getPublishTime(sinkRecord, nowInMillis); - long parsed = (publishTime / partitionDuration) * partitionDuration; - String timeString = dateTimeFormatter.format(Instant.ofEpochMilli(parsed).atOffset(ZoneOffset.UTC)); - final String result = timeString - + PATH_SEPARATOR - + getMessageOffset(sinkRecord); - return result; - } - - private long getPublishTime(Record sinkRecord, Long defaultTime) { - final Supplier defaultTimeSupplier = () -> { - LOGGER.warn("record not exist Message {}", sinkRecord.getRecordSequence().get()); - return defaultTime; - }; - return sinkRecord.getMessage() - .map(Message::getPublishTime) - .orElseGet(defaultTimeSupplier); - } -} diff --git a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/package-info.java b/src/main/java/org/apache/pulsar/io/jcloud/partitioner/package-info.java index 3d0aea09..c9c9c798 100644 --- a/src/main/java/org/apache/pulsar/io/jcloud/partitioner/package-info.java +++ b/src/main/java/org/apache/pulsar/io/jcloud/partitioner/package-info.java @@ -16,4 +16,4 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.io.jcloud.partitioner; +package org.apache.pulsar.io.jcloud.partitioner; \ No newline at end of file diff --git a/src/main/java/org/apache/pulsar/io/jcloud/sink/BlobStoreAbstractSink.java b/src/main/java/org/apache/pulsar/io/jcloud/sink/BlobStoreAbstractSink.java index 663e535b..36d3e837 100644 --- a/src/main/java/org/apache/pulsar/io/jcloud/sink/BlobStoreAbstractSink.java +++ b/src/main/java/org/apache/pulsar/io/jcloud/sink/BlobStoreAbstractSink.java @@ -19,44 +19,28 @@ package org.apache.pulsar.io.jcloud.sink; import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.pulsar.io.jcloud.batch.BatchManager.getBytesSum; import static org.apache.pulsar.io.jcloud.util.AvroRecordUtil.getPulsarSchema; -import com.google.common.collect.Lists; import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Optional; -import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.EnumUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.schema.GenericRecord; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.io.core.Sink; import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.jcloud.BlobStoreAbstractConfig; -import org.apache.pulsar.io.jcloud.format.AvroFormat; -import org.apache.pulsar.io.jcloud.format.BytesFormat; +import org.apache.pulsar.io.jcloud.batch.BatchManager; import org.apache.pulsar.io.jcloud.format.Format; -import org.apache.pulsar.io.jcloud.format.InitConfiguration; -import org.apache.pulsar.io.jcloud.format.JsonFormat; -import org.apache.pulsar.io.jcloud.format.ParquetFormat; -import org.apache.pulsar.io.jcloud.partitioner.LegacyPartitioner; -import org.apache.pulsar.io.jcloud.partitioner.PartitionerType; -import org.apache.pulsar.io.jcloud.partitioner.TopicPartitioner; -import org.apache.pulsar.io.jcloud.partitioner.legacy.LegacyPartitionerType; -import org.apache.pulsar.io.jcloud.partitioner.legacy.Partitioner; -import org.apache.pulsar.io.jcloud.partitioner.legacy.SimplePartitioner; -import org.apache.pulsar.io.jcloud.partitioner.legacy.TimePartitioner; +import org.apache.pulsar.io.jcloud.partitioner.Partitioner; import org.apache.pulsar.io.jcloud.writer.BlobWriter; import org.jclouds.blobstore.ContainerNotFoundException; @@ -67,140 +51,40 @@ @Slf4j public abstract class BlobStoreAbstractSink implements Sink { - private V sinkConfig; - - protected Partitioner legacyPartitioner; - protected org.apache.pulsar.io.jcloud.partitioner.Partitioner partitioner; - - protected Format format; + private static final String METRICS_TOTAL_SUCCESS = "_cloud_storage_sink_total_success_"; + private static final String METRICS_TOTAL_FAILURE = "_cloud_storage_sink_total_failure_"; + private static final String METRICS_LATEST_UPLOAD_ELAPSED_TIME = "_cloud_storage_latest_upload_elapsed_time_"; private final ScheduledExecutorService flushExecutor = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryBuilder() - .setNameFormat("pulsar-io-cloud-storage-sink-flush-%d") - .build());; - - private String pathPrefix; + .setNameFormat("pulsar-io-cloud-storage-sink-flush-%d") + .build());; - private long maxBatchSize; - private long maxBatchBytes; - final AtomicLong currentBatchSize = new AtomicLong(0L); - final AtomicLong currentBatchBytes = new AtomicLong(0L); - private ArrayBlockingQueue> pendingFlushQueue; - private final AtomicBoolean isFlushRunning = new AtomicBoolean(false); + protected Partitioner partitioner; + protected Format format; + protected BatchManager batchManager; + protected BlobWriter blobWriter; + private V sinkConfig; private SinkContext sinkContext; private volatile boolean isRunning = false; - - private BlobWriter blobWriter; - - private static final String METRICS_TOTAL_SUCCESS = "_cloud_storage_sink_total_success_"; - private static final String METRICS_TOTAL_FAILURE = "_cloud_storage_sink_total_failure_"; - private static final String METRICS_LATEST_UPLOAD_ELAPSED_TIME = "_cloud_storage_latest_upload_elapsed_time_"; + private final AtomicBoolean isFlushRunning = new AtomicBoolean(false); @Override public void open(Map config, SinkContext sinkContext) throws Exception { - sinkConfig = loadConfig(config, sinkContext); - sinkConfig.validate(); - pendingFlushQueue = new ArrayBlockingQueue<>(sinkConfig.getPendingQueueSize()); - format = buildFormat(sinkConfig); - if (format instanceof InitConfiguration) { - InitConfiguration formatConfigInitializer = - (InitConfiguration) format; - formatConfigInitializer.configure(sinkConfig); - } - partitioner = buildPartitioner(sinkConfig); - if (partitioner instanceof LegacyPartitioner) { - legacyPartitioner = buildLegacyPartitioner(sinkConfig); - } - pathPrefix = StringUtils.trimToEmpty(sinkConfig.getPathPrefix()); - long batchTimeMs = sinkConfig.getBatchTimeMs(); - maxBatchSize = sinkConfig.getBatchSize(); - maxBatchBytes = sinkConfig.getMaxBatchBytes(); - flushExecutor.scheduleWithFixedDelay(() -> this.flush(true), batchTimeMs, batchTimeMs, TimeUnit.MILLISECONDS); - isRunning = true; + this.sinkConfig = loadConfig(config, sinkContext); this.sinkContext = sinkContext; + this.format = Format.buildFormat(sinkConfig); + this.partitioner = Partitioner.buildPartitioner(sinkConfig); + this.isRunning = true; this.blobWriter = initBlobWriter(sinkConfig); - } - - private boolean isCurrentBatchThresholdReached() { - return currentBatchSize.get() >= maxBatchSize || currentBatchBytes.get() >= maxBatchBytes; - } - - private void flushIfNeeded(boolean force) { - if (isFlushRunning.get()) { - return; - } - if (force) { - flushExecutor.submit(() -> flush(true)); - } else if (isCurrentBatchThresholdReached()) { - flushExecutor.submit(() -> flush(false)); - } - } - - private Partitioner buildLegacyPartitioner(V sinkConfig) { - Partitioner partitioner; - String partitionerTypeName = sinkConfig.getPartitionerType(); - LegacyPartitionerType partitionerType = - EnumUtils.getEnumIgnoreCase(LegacyPartitionerType.class, partitionerTypeName, - LegacyPartitionerType.PARTITION); - switch (partitionerType) { - case TIME: - partitioner = new TimePartitioner<>(); - break; - case PARTITION: - default: - partitioner = new SimplePartitioner<>(); - break; - } - partitioner.configure(sinkConfig); - return partitioner; - } - - private org.apache.pulsar.io.jcloud.partitioner.Partitioner buildPartitioner(V sinkConfig) { - PartitionerType partitionerType = sinkConfig.getPartitioner(); - switch (partitionerType) { - case TOPIC: - return new TopicPartitioner(); - case TIME: - return new org.apache.pulsar.io.jcloud.partitioner.TimePartitioner(); - default: - return new LegacyPartitioner(); - } - } - - private Format buildFormat(V sinkConfig) { - String formatType = StringUtils.defaultIfBlank(sinkConfig.getFormatType(), "json"); - switch (formatType) { - case "avro": - return new AvroFormat(); - case "parquet": - return new ParquetFormat(); - case "json": - return new JsonFormat(); - case "bytes": - return new BytesFormat(); - default: - throw new RuntimeException("not support formatType " + formatType); - } + this.batchManager = BatchManager.createBatchManager(sinkConfig); + flushExecutor.scheduleWithFixedDelay(this::flush, sinkConfig.getBatchTimeMs() / 2 , + sinkConfig.getBatchTimeMs() / 2, TimeUnit.MILLISECONDS); } protected abstract V loadConfig(Map config, SinkContext sinkContext) throws IOException; protected abstract BlobWriter initBlobWriter(V sinkConfig); - public void uploadPayload(ByteBuffer payload, String filepath) throws IOException { - blobWriter.uploadBlob(filepath, payload); - } - - @Override - public void close() throws Exception { - isRunning = false; - flushIfNeeded(true); - flushExecutor.shutdown(); - if (!flushExecutor.awaitTermination(10 * sinkConfig.getBatchTimeMs(), TimeUnit.MILLISECONDS)) { - log.error("flushExecutor did not terminate in {} ms", 10 * sinkConfig.getBatchTimeMs()); - } - blobWriter.close(); - } - @Override public void write(Record record) throws Exception { if (log.isDebugEnabled()) { @@ -214,26 +98,33 @@ public void write(Record record) throws Exception { } checkArgument(record.getMessage().isPresent()); - pendingFlushQueue.put(record); - currentBatchSize.addAndGet(1); - currentBatchBytes.addAndGet(record.getMessage().get().size()); - flushIfNeeded(false); + batchManager.add(record); + flushIfNeeded(); } - - private void flush(boolean force) { - if (log.isDebugEnabled()) { - log.debug("flush requested, pending: {} ({} bytes), batchSize: {}, maxBatchBytes: {}", - currentBatchSize.get(), currentBatchBytes.get(), maxBatchSize, maxBatchBytes); + @Override + public void close() throws Exception { + isRunning = false; + flushExecutor.shutdown(); + if (!flushExecutor.awaitTermination(10 * sinkConfig.getBatchTimeMs(), TimeUnit.MILLISECONDS)) { + log.error("flushExecutor did not terminate in {} ms", 10 * sinkConfig.getBatchTimeMs()); } + blobWriter.close(); + } - if (pendingFlushQueue.isEmpty()) { - log.debug("Skip flushing because the pending flush queue is empty..."); + private void flushIfNeeded() { + if (isFlushRunning.get()) { return; } + if (batchManager.needFlush()) { + flushExecutor.submit(this::flush); + } + } + + private void flush() { - if (!force && !isCurrentBatchThresholdReached()) { - log.debug("Skip flushing because the batch is not full."); + if (batchManager.isEmpty()) { + log.debug("Skip flushing because the pending flush queue is empty..."); return; } @@ -249,35 +140,14 @@ private void flush(boolean force) { } finally { isFlushRunning.compareAndSet(true, false); } - flushIfNeeded(false); + flushIfNeeded(); } private void unsafeFlush() { - final List> recordsToInsert = Lists.newArrayList(); - long recordsToInsertBytes = 0; - while (!pendingFlushQueue.isEmpty() && recordsToInsert.size() < maxBatchSize - && recordsToInsertBytes < maxBatchBytes) { - Record r = pendingFlushQueue.poll(); - if (r != null) { - if (r.getMessage().isPresent()) { - long recordBytes = r.getMessage().get().size(); - recordsToInsertBytes += recordBytes; - } - recordsToInsert.add(r); - } - } - currentBatchBytes.addAndGet(-1 * recordsToInsertBytes); - currentBatchSize.addAndGet(-1 * recordsToInsert.size()); - log.info("Flushing {} buffered records to blob store. recordsToInsertBytes = {}", recordsToInsert.size(), - recordsToInsertBytes); - if (log.isDebugEnabled()) { - log.debug("buffered records {}", recordsToInsert); - } - - final Map>> recordsToInsertByTopic = - partitioner.partition(recordsToInsert); - + final long timeStampForPartitioning = System.currentTimeMillis(); + Map>> recordsToInsertByTopic = batchManager.pollNeedFlushData(); for (Map.Entry>> entry : recordsToInsertByTopic.entrySet()) { + String topicName = entry.getKey(); List> singleTopicRecordsToInsert = entry.getValue(); Record firstRecord = singleTopicRecordsToInsert.get(0); Schema schema; @@ -295,36 +165,18 @@ private void unsafeFlush() { return; } - String filepath; - try { - if (partitioner instanceof LegacyPartitioner) { - // all output blobs of the same batch should have the same partitioning timestamp - final long timeStampForPartitioning = System.currentTimeMillis(); - filepath = ((LegacyPartitioner) partitioner).buildPartitionPath(firstRecord, pathPrefix, - legacyPartitioner, format, timeStampForPartitioning); - } else { - filepath = pathPrefix + entry.getKey() + format.getExtension(); - } - } catch (Exception e) { - log.error("Failed to generate file path", e); - bulkHandleFailedRecords(singleTopicRecordsToInsert); - return; - } - + String filepath = ""; try { format.initSchema(schema); final Iterator> iter = singleTopicRecordsToInsert.iterator(); - + filepath = buildPartitionPath(firstRecord, partitioner, format, timeStampForPartitioning); ByteBuffer payload = bindValue(iter, format); int uploadSize = singleTopicRecordsToInsert.size(); long uploadBytes = getBytesSum(singleTopicRecordsToInsert); - log.info("Uploading blob {} from partition {} uploadSize {} out of currentBatchSize {} " - + " uploadBytes {} out of currentBatchBytes {}", - filepath, entry.getKey(), - uploadSize, currentBatchSize.get(), - uploadBytes, currentBatchBytes.get()); + log.info("Uploading blob {} from topic {} uploadSize:{} uploadBytes:{} currentBatchStatus:{}", + filepath, topicName, uploadSize, uploadBytes, batchManager.getCurrentStatsStr()); long elapsedMs = System.currentTimeMillis(); - uploadPayload(payload, filepath); + blobWriter.uploadBlob(filepath, payload); elapsedMs = System.currentTimeMillis() - elapsedMs; log.debug("Uploading blob {} elapsed time in ms: {}", filepath, elapsedMs); singleTopicRecordsToInsert.forEach(Record::ack); @@ -332,7 +184,7 @@ private void unsafeFlush() { sinkContext.recordMetric(METRICS_TOTAL_SUCCESS, singleTopicRecordsToInsert.size()); sinkContext.recordMetric(METRICS_LATEST_UPLOAD_ELAPSED_TIME, elapsedMs); } - log.info("Successfully uploaded blob {} from partition {} uploadSize {} uploadBytes {}", + log.info("Successfully uploaded blob {} from topic {} uploadSize {} uploadBytes {}", filepath, entry.getKey(), uploadSize, uploadBytes); } catch (Exception e) { @@ -356,9 +208,7 @@ private void bulkHandleFailedRecords(List> failedRecords) } else { failedRecords.forEach(Record::fail); } - if (sinkContext != null) { - sinkContext.recordMetric(METRICS_TOTAL_FAILURE, failedRecords.size()); - } + sinkContext.recordMetric(METRICS_TOTAL_FAILURE, failedRecords.size()); } public ByteBuffer bindValue(Iterator> message, @@ -366,13 +216,15 @@ public ByteBuffer bindValue(Iterator> message, return format.recordWriterBuf(message); } - private long getBytesSum(List> records) { - return records.stream() - .map(Record::getMessage) - .filter(Optional::isPresent) - .map(Optional::get) - .mapToLong(Message::size) - .sum(); - } + public String buildPartitionPath(Record message, + Partitioner partitioner, + Format format, + long partitioningTimestamp) { + String encodePartition = partitioner.encodePartition(message, partitioningTimestamp); + String partitionedPath = partitioner.generatePartitionedPath(message.getTopicName().get(), encodePartition); + String path = sinkConfig.getPathPrefix() + partitionedPath + format.getExtension(); + log.info("generate message[recordSequence={}] savePath: {}", message.getRecordSequence().get(), path); + return path; + } } diff --git a/src/main/java/org/apache/pulsar/io/jcloud/sink/CloudStorageGenericRecordSink.java b/src/main/java/org/apache/pulsar/io/jcloud/sink/CloudStorageGenericRecordSink.java index ed585ef3..fe6bcabc 100644 --- a/src/main/java/org/apache/pulsar/io/jcloud/sink/CloudStorageGenericRecordSink.java +++ b/src/main/java/org/apache/pulsar/io/jcloud/sink/CloudStorageGenericRecordSink.java @@ -46,7 +46,10 @@ public class CloudStorageGenericRecordSink extends BlobStoreAbstractSink config, SinkContext sinkContext) throws IOException { - return IOConfigUtils.loadWithSecrets(config, CloudStorageSinkConfig.class, sinkContext); + CloudStorageSinkConfig sinkConfig = + IOConfigUtils.loadWithSecrets(config, CloudStorageSinkConfig.class, sinkContext); + sinkConfig.validate(); + return sinkConfig; } @Override diff --git a/src/main/java/org/apache/pulsar/io/jcloud/BytesOutputStream.java b/src/main/java/org/apache/pulsar/io/jcloud/util/BytesOutputStream.java similarity index 98% rename from src/main/java/org/apache/pulsar/io/jcloud/BytesOutputStream.java rename to src/main/java/org/apache/pulsar/io/jcloud/util/BytesOutputStream.java index e13aa859..429ed5b8 100644 --- a/src/main/java/org/apache/pulsar/io/jcloud/BytesOutputStream.java +++ b/src/main/java/org/apache/pulsar/io/jcloud/util/BytesOutputStream.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.io.jcloud; +package org.apache.pulsar.io.jcloud.util; import java.io.IOException; import org.apache.commons.io.output.ByteArrayOutputStream; diff --git a/src/test/java/org/apache/pulsar/io/jcloud/ConnectorConfigTest.java b/src/test/java/org/apache/pulsar/io/jcloud/BlobStoreAbstractConfigTest.java similarity index 87% rename from src/test/java/org/apache/pulsar/io/jcloud/ConnectorConfigTest.java rename to src/test/java/org/apache/pulsar/io/jcloud/BlobStoreAbstractConfigTest.java index ac55f914..e9694b93 100644 --- a/src/test/java/org/apache/pulsar/io/jcloud/ConnectorConfigTest.java +++ b/src/test/java/org/apache/pulsar/io/jcloud/BlobStoreAbstractConfigTest.java @@ -20,12 +20,13 @@ import static org.apache.pulsar.io.jcloud.BlobStoreAbstractConfig.PROVIDER_AWSS3; import static org.apache.pulsar.io.jcloud.BlobStoreAbstractConfig.PROVIDER_AZURE; +import static org.apache.pulsar.io.jcloud.BlobStoreAbstractConfig.PROVIDER_GCS; import java.io.IOException; import java.util.HashMap; import java.util.Map; import org.apache.pulsar.io.common.IOConfigUtils; import org.apache.pulsar.io.core.SinkContext; -import org.apache.pulsar.io.jcloud.partitioner.legacy.LegacyPartitionerType; +import org.apache.pulsar.io.jcloud.partitioner.PartitionerType; import org.apache.pulsar.io.jcloud.sink.CloudStorageSinkConfig; import org.junit.Assert; import org.junit.Test; @@ -34,7 +35,7 @@ /** * connector config test. */ -public class ConnectorConfigTest { +public class BlobStoreAbstractConfigTest { @Test public void loadBasicConfigTest() throws IOException { @@ -51,7 +52,6 @@ public void loadBasicConfigTest() throws IOException { config.put("timePartitionPattern", "yyyy-MM-dd"); config.put("timePartitionDuration", "2d"); config.put("batchSize", 10); - config.put("partitioner", "topic"); CloudStorageSinkConfig cloudStorageSinkConfig = CloudStorageSinkConfig.load(config); cloudStorageSinkConfig.validate(); @@ -65,8 +65,6 @@ public void loadBasicConfigTest() throws IOException { Assert.assertEquals(config.get("timePartitionPattern"), cloudStorageSinkConfig.getTimePartitionPattern()); Assert.assertEquals(config.get("timePartitionDuration"), cloudStorageSinkConfig.getTimePartitionDuration()); Assert.assertEquals(config.get("batchSize"), cloudStorageSinkConfig.getBatchSize()); - Assert.assertEquals(config.get("partitioner"), - cloudStorageSinkConfig.getPartitioner().toString().toLowerCase()); Assert.assertEquals((int) config.get("batchSize"), cloudStorageSinkConfig.getPendingQueueSize()); Assert.assertEquals(10000000L, cloudStorageSinkConfig.getMaxBatchBytes()); } @@ -292,7 +290,7 @@ public void testEmptyPartitionerType() throws IOException { config.put("partitionerType", "default"); cloudStorageSinkConfig = CloudStorageSinkConfig.load(config); cloudStorageSinkConfig.validate(); - for (LegacyPartitionerType value : LegacyPartitionerType.values()) { + for (PartitionerType value : PartitionerType.values()) { config.put("partitionerType", value); cloudStorageSinkConfig = CloudStorageSinkConfig.load(config); cloudStorageSinkConfig.validate(); @@ -362,4 +360,35 @@ public void testCodec() throws IOException { Assert.assertNull(cloudStorageSinkConfig.getParquetCodec()); } + @Test + public void loadGoogleCloudStorageProviderTest() throws IOException { + Map config = new HashMap<>(); + config.put("provider", PROVIDER_GCS); + config.put("gcsServiceAccountKeyFilePath", "/tmp/gcs.json"); + config.put("bucket", "testbucket"); + config.put("region", "localhost"); + config.put("endpoint", "https://us-standard"); + config.put("pathPrefix", "pulsar/"); + config.put("formatType", "avro"); + config.put("partitionerType", "default"); + config.put("timePartitionPattern", "yyyy-MM-dd"); + config.put("timePartitionDuration", "2d"); + config.put("batchSize", 10); + config.put("maxBatchBytes", 10000L); + CloudStorageSinkConfig cloudStorageSinkConfig = CloudStorageSinkConfig.load(config); + cloudStorageSinkConfig.validate(); + + Assert.assertEquals(PROVIDER_GCS, cloudStorageSinkConfig.getProvider()); + Assert.assertEquals(config.get("gcsServiceAccountKeyFilePath"), + cloudStorageSinkConfig.getGcsServiceAccountKeyFilePath()); + Assert.assertEquals(config.get("bucket"), cloudStorageSinkConfig.getBucket()); + Assert.assertEquals(config.get("region"), cloudStorageSinkConfig.getRegion()); + Assert.assertEquals(config.get("formatType"), cloudStorageSinkConfig.getFormatType()); + Assert.assertEquals(config.get("partitionerType"), cloudStorageSinkConfig.getPartitionerType()); + Assert.assertEquals(config.get("timePartitionPattern"), cloudStorageSinkConfig.getTimePartitionPattern()); + Assert.assertEquals(config.get("timePartitionDuration"), cloudStorageSinkConfig.getTimePartitionDuration()); + Assert.assertEquals(config.get("batchSize"), cloudStorageSinkConfig.getBatchSize()); + Assert.assertEquals(config.get("maxBatchBytes"), cloudStorageSinkConfig.getMaxBatchBytes()); + } + } diff --git a/src/test/java/org/apache/pulsar/io/jcloud/container/PulsarContainer.java b/src/test/java/org/apache/pulsar/io/jcloud/PulsarContainer.java similarity index 98% rename from src/test/java/org/apache/pulsar/io/jcloud/container/PulsarContainer.java rename to src/test/java/org/apache/pulsar/io/jcloud/PulsarContainer.java index 4c67b6fb..40edb428 100644 --- a/src/test/java/org/apache/pulsar/io/jcloud/container/PulsarContainer.java +++ b/src/test/java/org/apache/pulsar/io/jcloud/PulsarContainer.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.io.jcloud.container; +package org.apache.pulsar.io.jcloud; import org.testcontainers.containers.GenericContainer; import org.testcontainers.containers.wait.strategy.Wait; diff --git a/src/test/java/org/apache/pulsar/io/jcloud/batch/BatchContainerTest.java b/src/test/java/org/apache/pulsar/io/jcloud/batch/BatchContainerTest.java new file mode 100644 index 00000000..7db8651b --- /dev/null +++ b/src/test/java/org/apache/pulsar/io/jcloud/batch/BatchContainerTest.java @@ -0,0 +1,94 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.io.jcloud.batch; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import java.util.List; +import java.util.Optional; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.functions.api.Record; +import org.junit.Before; +import org.junit.Test; + +public class BatchContainerTest { + + private BatchContainer batchContainer; + private static final long MAX_BATCH_SIZE = 5; + private static final long MAX_BATCH_BYTES = 100; + private static final long MAX_BATCH_TIME_MS = 1000; + + @Before + public void setUp() { + batchContainer = new BatchContainer(MAX_BATCH_SIZE, MAX_BATCH_BYTES, MAX_BATCH_TIME_MS, 10); + } + + @Test + public void testAddAndFlushBySize() throws InterruptedException { + for (int i = 0; i < MAX_BATCH_SIZE; i++) { + batchContainer.add(createMockRecord(10)); + } + assertTrue(batchContainer.needFlush()); + List> records = batchContainer.pollNeedFlushRecords(); + assertEquals(MAX_BATCH_SIZE, records.size()); + assertTrue(batchContainer.isEmpty()); + } + + @Test + public void testAddAndFlushByBytes() throws InterruptedException { + for (int i = 0; i < 3; i++) { + batchContainer.add(createMockRecord(40)); + } + assertTrue(batchContainer.needFlush()); + List> records = batchContainer.pollNeedFlushRecords(); + assertEquals(3, records.size()); + assertTrue(batchContainer.isEmpty()); + } + + @Test + public void testFlushByTime() throws InterruptedException { + batchContainer.add(createMockRecord(10)); + Thread.sleep(MAX_BATCH_TIME_MS + 100); // Wait longer than maxBatchTimeMs + assertTrue(batchContainer.needFlush()); + List> records = batchContainer.pollNeedFlushRecords(); + assertEquals(1, records.size()); + assertTrue(batchContainer.isEmpty()); + } + + @Test + public void testPollData() throws InterruptedException { + batchContainer.add(createMockRecord(1)); + assertFalse(batchContainer.needFlush()); + List> records = batchContainer.pollNeedFlushRecords(); + assertEquals(1, records.size()); + assertTrue(batchContainer.isEmpty()); + } + + Record createMockRecord(int size) { + Message msg = mock(Message.class); + when(msg.size()).thenReturn(size); + Record mockRecord = mock(Record.class); + when(mockRecord.getMessage()).thenReturn(Optional.of(msg)); + return mockRecord; + } +} diff --git a/src/test/java/org/apache/pulsar/io/jcloud/batch/BlendBatchMangerTest.java b/src/test/java/org/apache/pulsar/io/jcloud/batch/BlendBatchMangerTest.java new file mode 100644 index 00000000..fbe0eddc --- /dev/null +++ b/src/test/java/org/apache/pulsar/io/jcloud/batch/BlendBatchMangerTest.java @@ -0,0 +1,112 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.io.jcloud.batch; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.functions.api.Record; +import org.junit.Test; + +public class BlendBatchMangerTest { + + public void test(long maxBatchSize, long maxBatchBytes, int maxPendingQueueSize) throws InterruptedException { + BlendBatchManager blendBatchManger = new BlendBatchManager(maxBatchSize, + maxBatchBytes, 10000, maxPendingQueueSize); + + for (int i = 0; i < 15; i++) { + if (i % 2 == 0) { + blendBatchManger.add(getRecord("topic-0", 2)); + } else { + blendBatchManger.add(getRecord("topic-1", 2)); + } + } + + // assert size and bytes + assertEquals(15, blendBatchManger.getCurrentBatchSize(null)); + assertEquals(30, blendBatchManger.getCurrentBatchBytes(null)); + + // assert trigger flush, and each topic records num is 5 + assertTrue(blendBatchManger.needFlush()); + Map>> flushData = blendBatchManger.pollNeedFlushData(); + assertEquals(2, flushData.size()); + assertEquals(5, flushData.get("topic-0").size()); + assertEquals(5, flushData.get("topic-1").size()); + assertFalse(blendBatchManger.isEmpty()); + assertEquals(5, blendBatchManger.getCurrentBatchSize(null)); + assertEquals(10, blendBatchManger.getCurrentBatchBytes(null)); + + // assert not need flush + assertFalse(blendBatchManger.needFlush()); + assertFalse(blendBatchManger.isEmpty()); + } + + @Test + public void testFlushBySize() throws InterruptedException { + test(10, 10000, 1000); + } + + @Test + public void testFlushByByteSize() throws InterruptedException { + test(10000, 20, 1000); + } + + @Test + public void testFlushByTimout() throws InterruptedException { + long maxBatchTimeout = 1000; + BlendBatchManager blendBatchManger = new BlendBatchManager(1000, + 1000, maxBatchTimeout, 1000); + + blendBatchManger.add(getRecord("topic-0", 2)); + blendBatchManger.add(getRecord("topic-1", 2)); + assertEquals(2, blendBatchManger.getCurrentBatchSize(null)); + assertEquals(4, blendBatchManger.getCurrentBatchBytes(null)); + Thread.sleep(maxBatchTimeout + 100); + + // Time out flush + Map>> flushData = blendBatchManger.pollNeedFlushData(); + assertEquals(2, flushData.size()); + assertEquals(1, flushData.get("topic-0").size()); + assertEquals(1, flushData.get("topic-1").size()); + assertTrue(blendBatchManger.isEmpty()); + assertEquals(0, blendBatchManger.getCurrentBatchSize(null)); + assertEquals(0, blendBatchManger.getCurrentBatchBytes(null)); + + // Time out again + Thread.sleep(maxBatchTimeout + 100); + assertTrue(blendBatchManger.pollNeedFlushData().isEmpty()); + } + + Record getRecord(String topicName, int size) { + Message msg = mock(Message.class); + when(msg.size()).thenReturn(size); + Record mockRecord = mock(Record.class); + when(mockRecord.getTopicName()).thenReturn(Optional.of(topicName)); + when(mockRecord.getMessage()).thenReturn(Optional.of(msg)); + return mockRecord; + } + +} \ No newline at end of file diff --git a/src/test/java/org/apache/pulsar/io/jcloud/batch/PartitionedBatchManagerTest.java b/src/test/java/org/apache/pulsar/io/jcloud/batch/PartitionedBatchManagerTest.java new file mode 100644 index 00000000..f7d7b408 --- /dev/null +++ b/src/test/java/org/apache/pulsar/io/jcloud/batch/PartitionedBatchManagerTest.java @@ -0,0 +1,136 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.io.jcloud.batch; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.functions.api.Record; +import org.junit.Test; + +public class PartitionedBatchManagerTest { + + public void test(long maxBatchSize, long maxBatchBytes, int maxPendingQueueSize) throws InterruptedException { + PartitionedBatchManager partitionedBatchManager = + new PartitionedBatchManager(maxBatchSize, maxBatchBytes, 10000, maxPendingQueueSize); + + for (int i = 0; i < 10; i++) { + if (i % 2 == 0) { + partitionedBatchManager.add(getRecord("topic-0", 2)); + } else { + partitionedBatchManager.add(getRecord("topic-1", 2)); + } + } + // assert not trigger flush by each topic records. + assertFalse(partitionedBatchManager.needFlush()); + Map>> flushData = partitionedBatchManager.pollNeedFlushData(); + assertEquals(0, flushData.size()); + + // add more 5 records to topic-0, then trigger flush. + for (int i = 0; i < 5; i++) { + partitionedBatchManager.add(getRecord("topic-0", 2)); + } + assertTrue(partitionedBatchManager.needFlush()); + flushData = partitionedBatchManager.pollNeedFlushData(); + assertEquals(1, flushData.size()); + assertEquals(10, flushData.get("topic-0").size()); + + // assert topic-0 currentBatchSize and currentBatchBytes + assertEquals(0, partitionedBatchManager.getCurrentBatchSize("topic-0")); + assertEquals(0, partitionedBatchManager.getCurrentBatchBytes("topic-0")); + + // assert topic-1 currentBatchSize and currentBatchBytes + assertEquals(5, partitionedBatchManager.getCurrentBatchSize("topic-1")); + assertEquals(10, partitionedBatchManager.getCurrentBatchBytes("topic-1")); + + // assert not need flush + assertFalse(partitionedBatchManager.needFlush()); + assertFalse(partitionedBatchManager.isEmpty()); + } + + @Test + public void testFlushBySize() throws InterruptedException { + test(10, 10000, 1000); + } + + @Test + public void testFlushByByteSize() throws InterruptedException { + test(10000, 20, 1000); + } + + @Test + public void testFlushByTimout() throws InterruptedException { + long maxBatchTimeout = 2000; + PartitionedBatchManager partitionedBatchManager = new PartitionedBatchManager(1000, + 100, maxBatchTimeout, 1000); + + // 1. Add and assert status + partitionedBatchManager.add(getRecord("topic-0", 2)); + partitionedBatchManager.add(getRecord("topic-1", 101)); + + // 2. First sleep maxBatchTimeout / 2 + Thread.sleep(maxBatchTimeout / 2); + + // 3. Poll flush data, assert topic-1 data + Map>> flushData = partitionedBatchManager.pollNeedFlushData(); + assertEquals(1, flushData.size()); + assertFalse(flushData.containsKey("topic-0")); + assertEquals(1, flushData.get("topic-1").size()); + + // 4. write topic-1 data again, assert not need flush + partitionedBatchManager.add(getRecord("topic-1", 2)); + // Second sleep maxBatchTimeout / 2 + Thread.sleep(maxBatchTimeout / 2 + 100); + + // 5. assert topic-0 message timeout + flushData = partitionedBatchManager.pollNeedFlushData(); + assertEquals(1, flushData.size()); + assertEquals(1, flushData.get("topic-0").size()); + assertFalse(flushData.containsKey("topic-1")); + + // 6. Sleep assert can get topic-1 data + Thread.sleep(maxBatchTimeout / 2 + 100); + flushData = partitionedBatchManager.pollNeedFlushData(); + assertEquals(1, flushData.size()); + assertFalse(flushData.containsKey("topic-0")); + assertEquals(1, flushData.get("topic-1").size()); + assertTrue(partitionedBatchManager.isEmpty()); + + // Sleep and trigger timeout, and assert not data need flush + Thread.sleep(maxBatchTimeout + 100); + assertTrue(partitionedBatchManager.pollNeedFlushData().isEmpty()); + } + + Record getRecord(String topicName, int size) { + Message msg = mock(Message.class); + when(msg.size()).thenReturn(size); + Record mockRecord = mock(Record.class); + when(mockRecord.getTopicName()).thenReturn(Optional.of(topicName)); + when(mockRecord.getMessage()).thenReturn(Optional.of(msg)); + return mockRecord; + } + +} \ No newline at end of file diff --git a/src/test/java/org/apache/pulsar/io/jcloud/format/FormatTestBase.java b/src/test/java/org/apache/pulsar/io/jcloud/format/FormatTestBase.java index 9a51360d..365e3afe 100644 --- a/src/test/java/org/apache/pulsar/io/jcloud/format/FormatTestBase.java +++ b/src/test/java/org/apache/pulsar/io/jcloud/format/FormatTestBase.java @@ -47,8 +47,6 @@ import org.apache.pulsar.common.schema.KeyValueEncodingType; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.io.jcloud.BlobStoreAbstractConfig; -import org.apache.pulsar.io.jcloud.PulsarTestBase; -import org.apache.pulsar.io.jcloud.bo.TestRecord; import org.apache.pulsar.io.jcloud.schema.proto.Test.TestMessage; import org.apache.pulsar.io.jcloud.util.MetadataUtil; import org.junit.Assert; diff --git a/src/test/java/org/apache/pulsar/io/jcloud/format/JsonFormatMethodTest.java b/src/test/java/org/apache/pulsar/io/jcloud/format/JsonFormatMethodTest.java index 208a5367..6096c97c 100644 --- a/src/test/java/org/apache/pulsar/io/jcloud/format/JsonFormatMethodTest.java +++ b/src/test/java/org/apache/pulsar/io/jcloud/format/JsonFormatMethodTest.java @@ -28,7 +28,6 @@ import org.apache.pulsar.client.api.schema.GenericRecordBuilder; import org.apache.pulsar.client.impl.schema.generic.GenericJsonSchema; import org.apache.pulsar.io.jcloud.BlobStoreAbstractConfig; -import org.apache.pulsar.io.jcloud.bo.TestRecord; import org.junit.Assert; import org.junit.Test; diff --git a/src/test/java/org/apache/pulsar/io/jcloud/format/JsonFormatTest.java b/src/test/java/org/apache/pulsar/io/jcloud/format/JsonFormatTest.java index 1b058bca..a5844344 100644 --- a/src/test/java/org/apache/pulsar/io/jcloud/format/JsonFormatTest.java +++ b/src/test/java/org/apache/pulsar/io/jcloud/format/JsonFormatTest.java @@ -38,7 +38,6 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.functions.api.Record; -import org.apache.pulsar.io.jcloud.bo.TestRecord; import org.apache.pulsar.jcloud.shade.com.google.common.io.ByteSource; import org.junit.Assert; import org.junit.Test; diff --git a/src/test/java/org/apache/pulsar/io/jcloud/format/ParquetFormatTest.java b/src/test/java/org/apache/pulsar/io/jcloud/format/ParquetFormatTest.java index 7db47ffe..60d9b5af 100644 --- a/src/test/java/org/apache/pulsar/io/jcloud/format/ParquetFormatTest.java +++ b/src/test/java/org/apache/pulsar/io/jcloud/format/ParquetFormatTest.java @@ -37,7 +37,6 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.io.jcloud.schema.proto.Test; -import org.apache.pulsar.io.jcloud.support.ParquetInputFile; import org.apache.pulsar.jcloud.shade.com.google.common.io.ByteSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/src/test/java/org/apache/pulsar/io/jcloud/support/ParquetInputFile.java b/src/test/java/org/apache/pulsar/io/jcloud/format/ParquetInputFile.java similarity index 98% rename from src/test/java/org/apache/pulsar/io/jcloud/support/ParquetInputFile.java rename to src/test/java/org/apache/pulsar/io/jcloud/format/ParquetInputFile.java index cec76c1f..5a82e1bd 100644 --- a/src/test/java/org/apache/pulsar/io/jcloud/support/ParquetInputFile.java +++ b/src/test/java/org/apache/pulsar/io/jcloud/format/ParquetInputFile.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.io.jcloud.support; +package org.apache.pulsar.io.jcloud.format; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; diff --git a/src/test/java/org/apache/pulsar/io/jcloud/PulsarTestBase.java b/src/test/java/org/apache/pulsar/io/jcloud/format/PulsarTestBase.java similarity index 99% rename from src/test/java/org/apache/pulsar/io/jcloud/PulsarTestBase.java rename to src/test/java/org/apache/pulsar/io/jcloud/format/PulsarTestBase.java index b96c1fdd..750950ac 100644 --- a/src/test/java/org/apache/pulsar/io/jcloud/PulsarTestBase.java +++ b/src/test/java/org/apache/pulsar/io/jcloud/format/PulsarTestBase.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.io.jcloud; +package org.apache.pulsar.io.jcloud.format; import static java.time.temporal.ChronoUnit.SECONDS; import static org.testcontainers.containers.PulsarContainer.BROKER_HTTP_PORT; @@ -41,7 +41,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.schema.KeyValue; import org.apache.pulsar.common.schema.SchemaType; -import org.apache.pulsar.io.jcloud.container.PulsarContainer; +import org.apache.pulsar.io.jcloud.PulsarContainer; import org.junit.AfterClass; import org.junit.BeforeClass; import org.testcontainers.containers.output.Slf4jLogConsumer; diff --git a/src/test/java/org/apache/pulsar/io/jcloud/bo/TestRecord.java b/src/test/java/org/apache/pulsar/io/jcloud/format/TestRecord.java similarity index 96% rename from src/test/java/org/apache/pulsar/io/jcloud/bo/TestRecord.java rename to src/test/java/org/apache/pulsar/io/jcloud/format/TestRecord.java index 726b5875..8f80a692 100644 --- a/src/test/java/org/apache/pulsar/io/jcloud/bo/TestRecord.java +++ b/src/test/java/org/apache/pulsar/io/jcloud/format/TestRecord.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.io.jcloud.bo; +package org.apache.pulsar.io.jcloud.format; import lombok.AllArgsConstructor; import lombok.Data; diff --git a/src/test/java/org/apache/pulsar/io/jcloud/partitioner/PartitionerTest.java b/src/test/java/org/apache/pulsar/io/jcloud/partitioner/PartitionerTest.java index c5569bfa..21719ba9 100644 --- a/src/test/java/org/apache/pulsar/io/jcloud/partitioner/PartitionerTest.java +++ b/src/test/java/org/apache/pulsar/io/jcloud/partitioner/PartitionerTest.java @@ -29,9 +29,6 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.io.jcloud.BlobStoreAbstractConfig; -import org.apache.pulsar.io.jcloud.partitioner.legacy.Partitioner; -import org.apache.pulsar.io.jcloud.partitioner.legacy.SimplePartitioner; -import org.apache.pulsar.io.jcloud.partitioner.legacy.TimePartitioner; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -198,4 +195,4 @@ public void testGeneratePartitionedPath() { () -> MessageFormat.format("expected: {0}\nactual: {1}", expected, encodePartition); Assert.assertEquals(supplier.get(), expectedPartitionedPath, partitionedPath); } -} +} \ No newline at end of file diff --git a/src/test/java/org/apache/pulsar/io/jcloud/partitioner/LegacyPartitionerTypeTest.java b/src/test/java/org/apache/pulsar/io/jcloud/partitioner/PartitionerTypeTest.java similarity index 73% rename from src/test/java/org/apache/pulsar/io/jcloud/partitioner/LegacyPartitionerTypeTest.java rename to src/test/java/org/apache/pulsar/io/jcloud/partitioner/PartitionerTypeTest.java index 3651747d..f8f3ef29 100644 --- a/src/test/java/org/apache/pulsar/io/jcloud/partitioner/LegacyPartitionerTypeTest.java +++ b/src/test/java/org/apache/pulsar/io/jcloud/partitioner/PartitionerTypeTest.java @@ -20,17 +20,15 @@ import static org.junit.Assert.assertEquals; import org.apache.commons.lang3.EnumUtils; -import org.apache.pulsar.io.jcloud.partitioner.legacy.LegacyPartitionerType; import org.junit.Test; /** * partitionerType unit test. */ -public class LegacyPartitionerTypeTest { +public class PartitionerTypeTest { @Test public void testValueOf() { - assertEquals( - LegacyPartitionerType.PARTITION, EnumUtils.getEnumIgnoreCase(LegacyPartitionerType.class, "partition")); - assertEquals(LegacyPartitionerType.TIME, EnumUtils.getEnumIgnoreCase(LegacyPartitionerType.class, "time")); + assertEquals(PartitionerType.PARTITION, EnumUtils.getEnumIgnoreCase(PartitionerType.class, "partition")); + assertEquals(PartitionerType.TIME, EnumUtils.getEnumIgnoreCase(PartitionerType.class, "time")); } } diff --git a/src/test/java/org/apache/pulsar/io/jcloud/partitioner/SliceTopicPartitionPartitionerTest.java b/src/test/java/org/apache/pulsar/io/jcloud/partitioner/SliceTopicPartitionPartitionerTest.java index 686595bc..cd33bf29 100644 --- a/src/test/java/org/apache/pulsar/io/jcloud/partitioner/SliceTopicPartitionPartitionerTest.java +++ b/src/test/java/org/apache/pulsar/io/jcloud/partitioner/SliceTopicPartitionPartitionerTest.java @@ -29,9 +29,6 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.io.jcloud.BlobStoreAbstractConfig; -import org.apache.pulsar.io.jcloud.partitioner.legacy.Partitioner; -import org.apache.pulsar.io.jcloud.partitioner.legacy.SimplePartitioner; -import org.apache.pulsar.io.jcloud.partitioner.legacy.TimePartitioner; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -175,4 +172,4 @@ public void testGeneratePartitionedPath() { () -> MessageFormat.format("expected: {0}\nactual: {1}", expected, encodePartition); Assert.assertEquals(supplier.get(), expectedPartitionedPath, partitionedPath); } -} +} \ No newline at end of file diff --git a/src/test/java/org/apache/pulsar/io/jcloud/provider/GoogleCloudStorageProviderTest.java b/src/test/java/org/apache/pulsar/io/jcloud/provider/GoogleCloudStorageProviderTest.java deleted file mode 100644 index 84926a0c..00000000 --- a/src/test/java/org/apache/pulsar/io/jcloud/provider/GoogleCloudStorageProviderTest.java +++ /dev/null @@ -1,64 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.io.jcloud.provider; - -import static org.apache.pulsar.io.jcloud.BlobStoreAbstractConfig.PROVIDER_GCS; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import org.apache.pulsar.io.jcloud.sink.CloudStorageSinkConfig; -import org.junit.Assert; -import org.junit.Test; - -/** - * google cloud storage provider test. - */ -public class GoogleCloudStorageProviderTest { - @Test - public void loadBasicConfigTest() throws IOException { - Map config = new HashMap<>(); - config.put("provider", PROVIDER_GCS); - config.put("gcsServiceAccountKeyFilePath", "/tmp/gcs.json"); - config.put("bucket", "testbucket"); - config.put("region", "localhost"); - config.put("endpoint", "https://us-standard"); - config.put("pathPrefix", "pulsar/"); - config.put("formatType", "avro"); - config.put("partitionerType", "default"); - config.put("timePartitionPattern", "yyyy-MM-dd"); - config.put("timePartitionDuration", "2d"); - config.put("batchSize", 10); - config.put("maxBatchBytes", 10000L); - CloudStorageSinkConfig cloudStorageSinkConfig = CloudStorageSinkConfig.load(config); - cloudStorageSinkConfig.validate(); - - Assert.assertEquals(PROVIDER_GCS, cloudStorageSinkConfig.getProvider()); - Assert.assertEquals(config.get("gcsServiceAccountKeyFilePath"), - cloudStorageSinkConfig.getGcsServiceAccountKeyFilePath()); - Assert.assertEquals(config.get("bucket"), cloudStorageSinkConfig.getBucket()); - Assert.assertEquals(config.get("region"), cloudStorageSinkConfig.getRegion()); - Assert.assertEquals(config.get("formatType"), cloudStorageSinkConfig.getFormatType()); - Assert.assertEquals(config.get("partitionerType"), cloudStorageSinkConfig.getPartitionerType()); - Assert.assertEquals(config.get("timePartitionPattern"), cloudStorageSinkConfig.getTimePartitionPattern()); - Assert.assertEquals(config.get("timePartitionDuration"), cloudStorageSinkConfig.getTimePartitionDuration()); - Assert.assertEquals(config.get("batchSize"), cloudStorageSinkConfig.getBatchSize()); - Assert.assertEquals(config.get("maxBatchBytes"), cloudStorageSinkConfig.getMaxBatchBytes()); - } - -} diff --git a/src/test/java/org/apache/pulsar/io/jcloud/sink/CloudStorageGenericRecordSinkTest.java b/src/test/java/org/apache/pulsar/io/jcloud/sink/CloudStorageSinkBatchBlendTest.java similarity index 82% rename from src/test/java/org/apache/pulsar/io/jcloud/sink/CloudStorageGenericRecordSinkTest.java rename to src/test/java/org/apache/pulsar/io/jcloud/sink/CloudStorageSinkBatchBlendTest.java index 6bb142c9..8f22dc14 100644 --- a/src/test/java/org/apache/pulsar/io/jcloud/sink/CloudStorageGenericRecordSinkTest.java +++ b/src/test/java/org/apache/pulsar/io/jcloud/sink/CloudStorageSinkBatchBlendTest.java @@ -20,7 +20,6 @@ import static org.awaitility.Awaitility.await; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doReturn; @@ -36,6 +35,7 @@ import java.util.Iterator; import java.util.Map; import java.util.Optional; +import java.util.concurrent.ThreadLocalRandom; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.schema.GenericRecord; @@ -46,6 +46,7 @@ import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.jcloud.format.Format; +import org.apache.pulsar.io.jcloud.partitioner.Partitioner; import org.apache.pulsar.io.jcloud.writer.BlobWriter; import org.junit.After; import org.junit.Assert; @@ -57,7 +58,7 @@ /** * Test for {@link CloudStorageGenericRecordSink}. */ -public class CloudStorageGenericRecordSinkTest { +public class CloudStorageSinkBatchBlendTest { private static final int PAYLOAD_BYTES = 100; @@ -83,19 +84,21 @@ public void setup() throws Exception { this.config.put("bucket", "just/a/test"); this.config.put("formatType", "bytes"); this.config.put("partitionerType", "default"); + this.config.put("batchModel", "BLEND"); this.sink = spy(new CloudStorageGenericRecordSink()); this.mockSinkContext = mock(SinkContext.class); this.mockBlobWriter = mock(BlobWriter.class); this.mockRecord = mock(Record.class); + doReturn("a/test.json").when(sink) + .buildPartitionPath(any(Record.class), any(Partitioner.class), any(Format.class), any(Long.class)); doReturn(mockBlobWriter).when(sink).initBlobWriter(any(CloudStorageSinkConfig.class)); doReturn(ByteBuffer.wrap(new byte[]{0x0})).when(sink).bindValue(any(Iterator.class), any(Format.class)); Message mockMessage = mock(Message.class); when(mockMessage.size()).thenReturn(PAYLOAD_BYTES); - GenericSchema schema = createTestSchema(); GenericRecord genericRecord = spy(createTestRecord(schema)); doReturn(new byte[]{0x1}).when(genericRecord).getSchemaVersion(); @@ -104,7 +107,6 @@ public void setup() throws Exception { when(mockRecord.getValue()).thenReturn(genericRecord); when(mockRecord.getSchema()).thenAnswer((Answer) invocationOnMock -> schema); when(mockRecord.getMessage()).thenReturn(Optional.of(mockMessage)); - when(mockRecord.getRecordSequence()).thenReturn(Optional.of(1L)); } @After @@ -159,6 +161,33 @@ public void repeatedlyFlushOnMaxBatchBytesTest() throws Exception { verifyRecordAck(100); } + @Test + @SuppressWarnings({"unchecked", "rawtypes"}) + public void repeatedlyFlushOnMultiConditionTest() throws Exception { + this.config.put("pendingQueueSize", 100); // accept high number of messages + this.config.put("batchTimeMs", 1000); + this.config.put("maxBatchBytes", 10 * PAYLOAD_BYTES); + this.config.put("batchSize", 5); + this.sink.open(this.config, this.mockSinkContext); + + // Gen random message size + Message randomMessage = mock(Message.class); + when(randomMessage.size()).thenAnswer((Answer) invocation -> { + int randomMultiplier = ThreadLocalRandom.current().nextInt(1, 6); + return PAYLOAD_BYTES * randomMultiplier; + }); + when(mockRecord.getMessage()).thenReturn(Optional.of(randomMessage)); + + int numberOfRecords = 100; + for (int i = 0; i < numberOfRecords; i++) { + this.sink.write(mockRecord); + Thread.sleep(ThreadLocalRandom.current().nextInt(1, 500)); + } + await().atMost(Duration.ofSeconds(60)).untilAsserted( + () -> verify(mockRecord, times(numberOfRecords)).ack() + ); + } + @Test public void testBatchCleanupWhenFlushCrashed() throws Exception { this.config.put("pendingQueueSize", 1000); @@ -171,46 +200,12 @@ public void testBatchCleanupWhenFlushCrashed() throws Exception { sendMockRecord(1); await().atMost(Duration.ofSeconds(10)).untilAsserted( () -> { - Assert.assertEquals(0, this.sink.currentBatchBytes.get()); - Assert.assertEquals(0, this.sink.currentBatchSize.get()); + Assert.assertEquals(0, this.sink.batchManager.getCurrentBatchBytes("test-topic")); + Assert.assertEquals(0, this.sink.batchManager.getCurrentBatchSize("test-topic")); } ); } - private void verifyPartitionerSinkFlush(String prefix) throws Exception { - this.sink.open(this.config, this.mockSinkContext); - - sendMockRecord(5); - await().atMost(Duration.ofSeconds(10)).untilAsserted( - () -> verify(mockBlobWriter, atLeastOnce()).uploadBlob( - argThat((String s) -> s.matches(prefix + "(\\d+)\\.json")), any(ByteBuffer.class)) - ); - } - - @Test - public void testTimePartitioner() throws Exception { - this.config.put("batchTimeMs", 60000); // set high batchTimeMs to prevent scheduled flush - this.config.put("maxBatchBytes", 10000); // set high maxBatchBytes to prevent flush - this.config.put("batchSize", 5); // force flush after 5 messages - this.config.put("pathPrefix", "time/"); - this.config.put("partitioner", "time"); - this.config.put("formatType", "json"); - - verifyPartitionerSinkFlush("time/"); - } - - @Test - public void testTopicPartitioner() throws Exception { - this.config.put("batchTimeMs", 60000); // set high batchTimeMs to prevent scheduled flush - this.config.put("maxBatchBytes", 10000); // set high maxBatchBytes to prevent flush - this.config.put("batchSize", 5); // force flush after 5 messages - this.config.put("pathPrefix", "topic/"); - this.config.put("partitioner", "topic"); - this.config.put("formatType", "json"); - - verifyPartitionerSinkFlush("topic/public/default/test-topic/"); - } - private void verifyRecordAck(int numberOfRecords) throws Exception { this.sink.open(this.config, this.mockSinkContext); sendMockRecord(numberOfRecords); diff --git a/src/test/java/org/apache/pulsar/io/jcloud/sink/CloudStorageSinkBatchPartitionedTest.java b/src/test/java/org/apache/pulsar/io/jcloud/sink/CloudStorageSinkBatchPartitionedTest.java new file mode 100644 index 00000000..e27fb936 --- /dev/null +++ b/src/test/java/org/apache/pulsar/io/jcloud/sink/CloudStorageSinkBatchPartitionedTest.java @@ -0,0 +1,326 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.io.jcloud.sink; + +import static org.awaitility.Awaitility.await; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.clearInvocations; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import java.nio.ByteBuffer; +import java.time.Duration; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.client.api.schema.GenericSchema; +import org.apache.pulsar.client.api.schema.RecordSchemaBuilder; +import org.apache.pulsar.client.api.schema.SchemaBuilder; +import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.functions.api.Record; +import org.apache.pulsar.io.core.SinkContext; +import org.apache.pulsar.io.jcloud.format.Format; +import org.apache.pulsar.io.jcloud.writer.BlobWriter; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.stubbing.Answer; + +/** + * Test for {@link CloudStorageGenericRecordSink}. + */ +public class CloudStorageSinkBatchPartitionedTest { + + private static final int PAYLOAD_BYTES = 100; + + @Mock + private SinkContext mockSinkContext; + + @Mock + private BlobWriter mockBlobWriter; + + @Mock + private Record mockRecordTopic1; + + @Mock + private Record mockRecordTopic2; + + private Map config; + + private CloudStorageGenericRecordSink sink; + + @Before + @SuppressWarnings({"unchecked", "rawtypes"}) + public void setup() throws Exception { + //initialize required parameters + this.config = new HashMap<>(); + this.config.put("provider", "google-cloud-storage"); + this.config.put("bucket", "just/a/test"); + this.config.put("formatType", "bytes"); + this.config.put("partitionerType", "PARTITION"); + this.config.put("batchModel", "PARTITIONED"); + + this.sink = spy(new CloudStorageGenericRecordSink()); + this.mockSinkContext = mock(SinkContext.class); + this.mockBlobWriter = mock(BlobWriter.class); + + doReturn(mockBlobWriter).when(sink).initBlobWriter(any(CloudStorageSinkConfig.class)); + doReturn(ByteBuffer.wrap(new byte[]{0x0})).when(sink).bindValue(any(Iterator.class), any(Format.class)); + + RecordSchemaBuilder schemaBuilder = SchemaBuilder.record("test"); + schemaBuilder.field("a").type(SchemaType.INT32).optional().defaultValue(null); + GenericSchema schema = Schema.generic(schemaBuilder.build(SchemaType.JSON)); + GenericRecord genericRecord = spy(schema.newRecordBuilder().set("a", 1).build()); + doReturn(new byte[]{0x1}).when(genericRecord).getSchemaVersion(); + + Message mockMessage = mock(Message.class); + when(mockMessage.size()).thenReturn(PAYLOAD_BYTES); + + this.mockRecordTopic1 = mock(Record.class); + when(mockRecordTopic1.getTopicName()).thenReturn(Optional.of("topic-1")); + when(mockRecordTopic1.getValue()).thenReturn(genericRecord); + when(mockRecordTopic1.getSchema()).thenAnswer((Answer) invocationOnMock -> schema); + when(mockRecordTopic1.getMessage()).thenReturn(Optional.of(mockMessage)); + when(mockRecordTopic1.getRecordSequence()).thenReturn(Optional.of(100L)); + + this.mockRecordTopic2 = mock(Record.class); + when(mockRecordTopic2.getTopicName()).thenReturn(Optional.of("topic-2")); + when(mockRecordTopic2.getValue()).thenReturn(genericRecord); + when(mockRecordTopic2.getSchema()).thenAnswer((Answer) invocationOnMock -> schema); + when(mockRecordTopic2.getMessage()).thenReturn(Optional.of(mockMessage)); + when(mockRecordTopic2.getRecordSequence()).thenReturn(Optional.of(200L)); + } + + @After + public void tearDown() throws Exception { + this.sink.close(); + } + + @Test + public void flushOnMaxBatchBytesTest() throws Exception { + this.config.put("batchTimeMs", 60000); // set high batchTimeMs to prevent scheduled flush + this.config.put("batchSize", 1000); // set high batchSize to prevent flush + this.config.put("maxBatchBytes", 5 * PAYLOAD_BYTES); // force flush after 500 bytes + verifySinkFlush(); + } + + @Test + public void flushOnBatchSizeTests() throws Exception { + this.config.put("batchTimeMs", 60000); // set high batchTimeMs to prevent scheduled flush + this.config.put("maxBatchBytes", 10000); // set high maxBatchBytes to prevent flush + this.config.put("batchSize", 5); // force flush after 5 messages + + verifySinkFlush(); + } + + @Test + public void flushOnTimeOutTests() throws Exception { + long maxBatchTimeout = 2000; + this.config.put("batchTimeMs", maxBatchTimeout); // set high batchTimeMs to prevent scheduled flush + this.config.put("maxBatchBytes", 100000); // set high maxBatchBytes to prevent flush + this.config.put("batchSize", 10); // force flush after 5 messages + + this.sink.open(this.config, this.mockSinkContext); + + // 0. Write 2 data for each topic + for (int i = 0; i < 2; i++) { + sink.write(mockRecordTopic1); + } + for (int i = 0; i < 2; i++) { + sink.write(mockRecordTopic2); + } + + // 1. First sleep maxBatchTimeout / 2, and not data need flush + Thread.sleep(maxBatchTimeout / 2); + verify(mockBlobWriter, never()).uploadBlob(any(String.class), any(ByteBuffer.class)); + + // 2. Write 8 for topic-1 and to trigger flush + for (int i = 0; i < 8; i++) { + sink.write(mockRecordTopic1); + } + await().atMost(Duration.ofSeconds(10)).pollInterval(Duration.ofMillis(10)).untilAsserted( + () -> verify(mockBlobWriter, times(1)) + .uploadBlob(eq("public/default/topic-1/100.raw"), any(ByteBuffer.class)) + ); + verify(mockBlobWriter, never()).uploadBlob(eq("public/default/topic-2/200.raw"), any(ByteBuffer.class)); + + // 3. Write 2 message for topic-1 again and assert not message need flush(no timeout) + for (int i = 0; i < 2; i++) { + sink.write(mockRecordTopic1); + } + clearInvocations(mockBlobWriter); + verify(mockBlobWriter, never()).uploadBlob(eq("public/default/topic-1/100.raw"), any(ByteBuffer.class)); + + // 4. Second sleep maxBatchTimeout / 2 again, and assert topic-2 data need flush + // and topic-1 no need flush(no timeout) + Thread.sleep(maxBatchTimeout / 2 + 100); + await().atMost(Duration.ofSeconds(10)).pollInterval(Duration.ofMillis(10)).untilAsserted( + () -> verify(mockBlobWriter, times(1)) + .uploadBlob(eq("public/default/topic-2/200.raw"), any(ByteBuffer.class)) + ); + verify(mockBlobWriter, never()).uploadBlob(eq("public/default/topic-1/100.raw"), any(ByteBuffer.class)); + + // 5. Assert for topic-1 flush data step-3 write data. + await().atMost(Duration.ofSeconds(10)).untilAsserted( + () -> verify(mockBlobWriter, times(1)) + .uploadBlob(eq("public/default/topic-1/100.raw"), any(ByteBuffer.class)) + ); + + // 6. Assert all message has been ack + await().atMost(Duration.ofSeconds(10)).untilAsserted( + () -> verify(mockRecordTopic1, times(12)).ack() + ); + await().atMost(Duration.ofSeconds(10)).untilAsserted( + () -> verify(mockRecordTopic2, times(2)).ack() + ); + } + + @Test + public void repeatedlyFlushOnBatchSizeTest() throws Exception { + this.config.put("pendingQueueSize", 1000); // accept high number of messages + this.config.put("batchTimeMs", 60000); // set high batchTimeMs to prevent scheduled flush + this.config.put("maxBatchBytes", 100000); // set high maxBatchBytes to prevent flush + this.config.put("batchSize", 5); // force flush after 5 messages + + verifyRecordAck(100, -1); + } + + @Test + public void repeatedlyFlushOnMaxBatchBytesTest() throws Exception { + this.config.put("pendingQueueSize", 1000); // accept high number of messages + this.config.put("batchTimeMs", 60000); // set high batchTimeMs to prevent scheduled flush + this.config.put("maxBatchBytes", 5 * PAYLOAD_BYTES); // force flush after 500 bytes + this.config.put("batchSize", 1000); // set high batchSize to prevent flush + + verifyRecordAck(100, -1); + } + + @Test + @SuppressWarnings({"unchecked", "rawtypes"}) + public void repeatedlyFlushOnMultiConditionTest() throws Exception { + this.config.put("pendingQueueSize", 100); + this.config.put("batchTimeMs", 1000); + this.config.put("maxBatchBytes", 10 * PAYLOAD_BYTES); + this.config.put("batchSize", 5); + this.sink.open(this.config, this.mockSinkContext); + + // Gen random message size + Message randomMessage = mock(Message.class); + when(randomMessage.size()).thenAnswer((Answer) invocation -> { + int randomMultiplier = ThreadLocalRandom.current().nextInt(1, 6); + return PAYLOAD_BYTES * randomMultiplier; + }); + when(mockRecordTopic1.getMessage()).thenReturn(Optional.of(randomMessage)); + when(mockRecordTopic2.getMessage()).thenReturn(Optional.of(randomMessage)); + + int numberOfRecords = 100; + for (int i = 0; i < numberOfRecords; i++) { + this.sink.write(mockRecordTopic1); + this.sink.write(mockRecordTopic2); + Thread.sleep(ThreadLocalRandom.current().nextInt(1, 500)); + } + await().atMost(Duration.ofSeconds(60)).untilAsserted( + () -> verify(mockRecordTopic1, times(numberOfRecords)).ack() + ); + await().atMost(Duration.ofSeconds(60)).untilAsserted( + () -> verify(mockRecordTopic2, times(numberOfRecords)).ack() + ); + } + + @Test + public void testBatchCleanupWhenFlushCrashed() throws Exception { + this.config.put("pendingQueueSize", 1000); + this.config.put("batchTimeMs", 1000); + this.config.put("maxBatchBytes", 5 * PAYLOAD_BYTES); + this.config.put("batchSize", 1); + + this.sink.open(this.config, this.mockSinkContext); + when(mockRecordTopic1.getSchema()).thenThrow(new OutOfMemoryError()); + when(mockRecordTopic2.getSchema()).thenThrow(new OutOfMemoryError()); + this.sink.write(mockRecordTopic1); + this.sink.write(mockRecordTopic2); + await().atMost(Duration.ofSeconds(10)).untilAsserted( + () -> { + Assert.assertEquals(0, this.sink.batchManager.getCurrentBatchBytes("topic-1")); + Assert.assertEquals(0, this.sink.batchManager.getCurrentBatchSize("topic-1")); + Assert.assertEquals(0, this.sink.batchManager.getCurrentBatchBytes("topic-2")); + Assert.assertEquals(0, this.sink.batchManager.getCurrentBatchSize("topic-2")); + } + ); + } + + private void verifyRecordAck(int numberOfRecords, long sleepMillis) throws Exception { + this.sink.open(this.config, this.mockSinkContext); + for (int i = 0; i < numberOfRecords; i++) { + this.sink.write(mockRecordTopic1); + this.sink.write(mockRecordTopic2); + if (sleepMillis > 0) { + Thread.sleep(sleepMillis); + } + } + await().atMost(Duration.ofSeconds(30)).untilAsserted( + () -> verify(mockRecordTopic1, times(numberOfRecords)).ack() + ); + await().atMost(Duration.ofSeconds(30)).untilAsserted( + () -> verify(mockRecordTopic2, times(numberOfRecords)).ack() + ); + } + + private void verifySinkFlush() throws Exception { + this.sink.open(this.config, this.mockSinkContext); + + for (int i = 0; i < 4; i++) { + this.sink.write(mockRecordTopic1); + } + verify(mockBlobWriter, never()).uploadBlob(any(String.class), any(ByteBuffer.class)); + + for (int i = 0; i < 5; i++) { + this.sink.write(mockRecordTopic2); + } + + await().atMost(Duration.ofSeconds(10)).untilAsserted( + () -> verify(mockBlobWriter, times(1)) + .uploadBlob(eq("public/default/topic-2/200.raw"), any(ByteBuffer.class)) + ); + await().atMost(Duration.ofSeconds(30)).untilAsserted( + () -> verify(mockRecordTopic2, times(5)).ack() + ); + + this.sink.write(mockRecordTopic1); + await().atMost(Duration.ofSeconds(10)).untilAsserted( + () -> verify(mockBlobWriter, times(1)) + .uploadBlob(eq("public/default/topic-1/100.raw"), any(ByteBuffer.class)) + ); + await().atMost(Duration.ofSeconds(30)).untilAsserted( + () -> verify(mockRecordTopic1, times(5)).ack() + ); + } +} diff --git a/src/test/java/org/apache/pulsar/io/jcloud/HexStringUtilsTest.java b/src/test/java/org/apache/pulsar/io/jcloud/utils/HexStringUtilsTest.java similarity index 97% rename from src/test/java/org/apache/pulsar/io/jcloud/HexStringUtilsTest.java rename to src/test/java/org/apache/pulsar/io/jcloud/utils/HexStringUtilsTest.java index 0c7106f9..05f27133 100644 --- a/src/test/java/org/apache/pulsar/io/jcloud/HexStringUtilsTest.java +++ b/src/test/java/org/apache/pulsar/io/jcloud/utils/HexStringUtilsTest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.io.jcloud; +package org.apache.pulsar.io.jcloud.utils; import java.io.IOException; import org.apache.pulsar.io.jcloud.util.HexStringUtils; diff --git a/src/test/java/org/apache/pulsar/io/jcloud/MetadataUtilTest.java b/src/test/java/org/apache/pulsar/io/jcloud/utils/MetadataUtilTest.java similarity index 98% rename from src/test/java/org/apache/pulsar/io/jcloud/MetadataUtilTest.java rename to src/test/java/org/apache/pulsar/io/jcloud/utils/MetadataUtilTest.java index 0c4f114a..251d26cb 100644 --- a/src/test/java/org/apache/pulsar/io/jcloud/MetadataUtilTest.java +++ b/src/test/java/org/apache/pulsar/io/jcloud/utils/MetadataUtilTest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.io.jcloud; +package org.apache.pulsar.io.jcloud.utils; import static org.apache.pulsar.io.jcloud.util.MetadataUtil.METADATA_MESSAGE_ID_KEY; import static org.apache.pulsar.io.jcloud.util.MetadataUtil.METADATA_SCHEMA_VERSION_KEY;