From 7103cb4b9df56f8ecc4a27672d34dc21da58b812 Mon Sep 17 00:00:00 2001 From: Tejaswini Bandlamudi <96047043+tejaswini-imply@users.noreply.github.com> Date: Thu, 2 Mar 2023 18:07:17 +0530 Subject: [PATCH] Removes FiniteFirehoseFactory and its implementations (#12852) The FiniteFirehoseFactory and InputRowParser classes were deprecated in 0.17.0 (#8823) in favor of InputSource & InputFormat. This PR removes the FiniteFirehoseFactory and all its implementations along with classes solely used by them like Fetcher (Used by PrefetchableTextFilesFirehoseFactory). Refactors classes including tests using FiniteFirehoseFactory to use InputSource instead. Removing InputRowParser may not be as trivial as many classes that aren't deprecated depends on it (with no alternatives), like EventReceiverFirehoseFactory. Hence FirehoseFactory, EventReceiverFirehoseFactory, and Firehose are marked deprecated. --- .github/workflows/reusable-revised-its.yml | 2 +- docs/configuration/index.md | 5 +- docs/development/extensions-core/google.md | 2 - docs/development/extensions-core/mysql.md | 34 +- .../development/extensions-core/postgresql.md | 34 +- docs/ingestion/native-batch.md | 4 +- .../aliyun/OssFirehoseDruidModule.java | 46 -- .../aliyun/StaticOssFirehoseFactory.java | 243 ------- ...rg.apache.druid.initialization.DruidModule | 1 - .../cloudfiles-extensions/pom.xml | 4 - .../firehose/cloudfiles/CloudFilesBlob.java | 101 --- .../CloudFilesFirehoseDruidModule.java | 48 -- .../StaticCloudFilesFirehoseFactory.java | 162 ----- ...rg.apache.druid.initialization.DruidModule | 1 - .../StaticCloudFilesFirehoseFactoryTest.java | 105 --- .../druid/firehose/azure/AzureBlob.java | 93 --- .../StaticAzureBlobStoreFirehoseFactory.java | 167 ----- .../azure/AzureStorageDruidModule.java | 2 - ...aticAzureBlobStoreFirehoseFactoryTest.java | 104 --- .../druid/firehose/google/GoogleBlob.java | 81 --- .../StaticGoogleBlobStoreFirehoseFactory.java | 155 ----- .../google/GoogleStorageDruidModule.java | 2 - ...ticGoogleBlobStoreFirehoseFactoryTest.java | 63 -- .../firehose/hdfs/HdfsFirehoseFactory.java | 127 ---- .../storage/hdfs/HdfsStorageDruidModule.java | 2 - .../hdfs/HdfsFirehoseFactoryTest.java | 218 ------ .../indexing/kafka/KafkaSamplerSpecTest.java | 90 ++- .../kinesis/KinesisSamplerSpecTest.java | 123 +++- .../firehose/s3/S3FirehoseDruidModule.java | 48 -- .../firehose/s3/StaticS3FirehoseFactory.java | 236 ------- ...rg.apache.druid.initialization.DruidModule | 1 - .../s3/StaticS3FirehoseFactoryTest.java | 115 ---- .../guice/IndexingServiceFirehoseModule.java | 4 +- .../common/ReingestionTimelineUtils.java | 140 ---- .../common/task/AbstractBatchIndexTask.java | 58 +- .../AppenderatorDriverRealtimeIndexTask.java | 1 + .../druid/indexing/common/task/IndexTask.java | 21 +- .../common/task/RealtimeIndexTask.java | 1 + ...putSourceSplitParallelIndexTaskRunner.java | 14 +- .../parallel/ParallelIndexIngestionSpec.java | 5 +- .../parallel/ParallelIndexSupervisorTask.java | 10 +- .../PartialDimensionCardinalityTask.java | 4 +- .../PartialDimensionDistributionTask.java | 4 +- .../parallel/PartialSegmentGenerateTask.java | 4 +- .../SinglePhaseParallelIndexTaskRunner.java | 15 +- .../batch/parallel/SinglePhaseSubTask.java | 7 +- .../IngestSegmentFirehoseFactory.java | 330 --------- .../sampler/IndexTaskSamplerSpec.java | 35 +- .../SeekableStreamSamplerSpec.java | 150 ++-- ...stRealtimeTask.java => TestIndexTask.java} | 54 +- .../common/task/CompactionTaskRunTest.java | 93 --- .../common/task/IndexIngestionSpecTest.java | 55 -- .../indexing/common/task/IndexTaskTest.java | 207 +++--- .../common/task/IngestionTestBase.java | 57 ++ .../indexing/common/task/TaskSerdeTest.java | 21 +- ...bstractMultiPhaseParallelIndexingTest.java | 20 +- ...stractParallelIndexSupervisorTaskTest.java | 6 +- .../parallel/HashPartitionTaskKillTest.java | 18 +- .../ParallelIndexSupervisorTaskSerdeTest.java | 5 - .../ParallelIndexSupervisorTaskTest.java | 81 +++ .../SinglePhaseParallelIndexingTest.java | 29 +- .../IngestSegmentFirehoseFactoryTest.java | 615 ----------------- ...estSegmentFirehoseFactoryTimelineTest.java | 453 ------------ .../overlord/RemoteTaskRunnerTest.java | 62 +- .../sampler/InputSourceSamplerTest.java | 492 +------------- .../SeekableStreamSamplerSpecTest.java | 354 ++++++++++ .../indexing/worker/TaskAnnouncementTest.java | 10 +- .../worker/WorkerTaskMonitorTest.java | 4 +- .../ITCombiningFirehoseFactoryIndexTest.java | 101 --- ...CombiningInputSourceParallelIndexTest.java | 2 +- .../testsEx/indexer/ITTransformTest.java | 54 -- ...kipedia_combining_firehose_index_task.json | 95 --- ...combining_input_source_index_queries.json} | 0 ...ipedia_index_task_with_dimension_spec.json | 59 +- ...edia_index_task_with_granularity_spec.json | 28 +- .../wikipedia_index_task_with_transform.json | 103 --- ...ia_index_with_merge_column_limit_task.json | 48 +- .../indexer/wikipedia_merge_index_task.json | 26 +- .../indexer/wikipedia_merge_reindex_task.json | 30 +- .../wikipedia_parallel_index_task.json | 48 +- ...ia_parallel_ingest_segment_index_task.json | 30 +- .../wikipedia_parallel_reindex_task.json | 48 +- .../indexer/wikipedia_reindex_task.json | 26 +- ...ikipedia_reindex_task_with_transforms.json | 108 --- .../wikipedia_with_timestamp_index_task.json | 61 +- .../indexer/wikiticker_index_task.json | 33 +- ...combining_input_source_index_queries.json} | 0 .../ITCombiningFirehoseFactoryIndexTest.java | 100 --- ...CombiningInputSourceParallelIndexTest.java | 2 +- .../druid/tests/indexer/ITTransformTest.java | 51 -- ...kipedia_combining_firehose_index_task.json | 95 --- ...combining_input_source_index_queries.json} | 0 ...ipedia_index_task_with_dimension_spec.json | 59 +- ...edia_index_task_with_granularity_spec.json | 28 +- .../wikipedia_index_task_with_transform.json | 103 --- ...ia_index_with_merge_column_limit_task.json | 48 +- .../indexer/wikipedia_merge_index_task.json | 26 +- .../indexer/wikipedia_merge_reindex_task.json | 30 +- .../wikipedia_parallel_index_task.json | 48 +- ...ia_parallel_ingest_segment_index_task.json | 30 +- .../wikipedia_parallel_reindex_task.json | 48 +- .../indexer/wikipedia_reindex_task.json | 26 +- ...ikipedia_reindex_task_with_transforms.json | 108 --- .../wikipedia_with_timestamp_index_task.json | 61 +- .../indexer/wikiticker_index_task.json | 33 +- .../data/input/FiniteFirehoseFactory.java | 72 -- .../org/apache/druid/data/input/Firehose.java | 1 + .../druid/data/input/FirehoseFactory.java | 19 +- .../FirehoseFactoryToInputSourceAdaptor.java | 99 --- .../druid/data/input/SplitHintSpec.java | 2 - .../AbstractTextFilesFirehoseFactory.java | 144 ---- .../input/impl/FileIteratingFirehose.java | 118 ---- .../FirehoseToInputSourceReaderAdaptor.java | 124 ---- .../data/input/impl/RetryingInputStream.java | 3 +- .../input/impl/prefetch/CacheManager.java | 97 --- .../data/input/impl/prefetch/FetchConfig.java | 96 --- .../data/input/impl/prefetch/FetchedFile.java | 68 -- .../data/input/impl/prefetch/Fetcher.java | 292 -------- .../data/input/impl/prefetch/FileFetcher.java | 111 --- .../impl/prefetch/ObjectOpenFunction.java | 7 - .../data/input/impl/prefetch/OpenObject.java | 71 -- .../PrefetchableTextFilesFirehoseFactory.java | 318 --------- .../segment/transform/TransformSpec.java | 15 - .../transform/TransformingInputRowParser.java | 5 - .../TransformingStringInputRowParser.java | 5 - ...rehoseFactoryToInputSourceAdaptorTest.java | 158 ----- .../input/impl/FileIteratingFirehoseTest.java | 171 ----- .../data/input/impl/NoopFirehoseFactory.java | 54 -- ...fetchableTextFilesFirehoseFactoryTest.java | 643 ------------------ .../apache/druid/guice/FirehoseModule.java | 12 +- .../firehose/ClippedFirehoseFactory.java | 1 + .../firehose/CombiningFirehoseFactory.java | 160 ----- .../EventReceiverFirehoseFactory.java | 1 + .../firehose/HttpFirehoseFactory.java | 165 ----- .../realtime/firehose/InlineFirehose.java | 91 --- .../firehose/InlineFirehoseFactory.java | 105 --- .../firehose/LocalFirehoseFactory.java | 120 ---- .../firehose/PrefetchSqlFirehoseFactory.java | 280 -------- .../segment/realtime/firehose/SqlFetcher.java | 101 --- .../realtime/firehose/SqlFirehose.java | 95 --- .../realtime/firehose/SqlFirehoseFactory.java | 115 ---- .../firehose/TimedShutoffFirehoseFactory.java | 1 + .../CombiningFirehoseFactoryTest.java | 174 ----- .../firehose/HttpFirehoseFactoryTest.java | 145 ---- .../firehose/InlineFirehoseFactoryTest.java | 129 ---- .../realtime/firehose/InlineFirehoseTest.java | 218 ------ .../firehose/LocalFirehoseFactoryTest.java | 105 --- .../firehose/SqlFirehoseFactoryTest.java | 182 ----- .../realtime/firehose/SqlFirehoseTest.java | 269 -------- .../cli/validate/DruidJsonValidatorTest.java | 54 +- 150 files changed, 1557 insertions(+), 11313 deletions(-) delete mode 100644 extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/firehose/aliyun/OssFirehoseDruidModule.java delete mode 100644 extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/firehose/aliyun/StaticOssFirehoseFactory.java delete mode 100644 extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesBlob.java delete mode 100644 extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesFirehoseDruidModule.java delete mode 100644 extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java delete mode 100644 extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactoryTest.java delete mode 100644 extensions-core/azure-extensions/src/main/java/org/apache/druid/firehose/azure/AzureBlob.java delete mode 100644 extensions-core/azure-extensions/src/main/java/org/apache/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java delete mode 100644 extensions-core/azure-extensions/src/test/java/org/apache/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactoryTest.java delete mode 100644 extensions-core/google-extensions/src/main/java/org/apache/druid/firehose/google/GoogleBlob.java delete mode 100644 extensions-core/google-extensions/src/main/java/org/apache/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java delete mode 100644 extensions-core/google-extensions/src/test/java/org/apache/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactoryTest.java delete mode 100644 extensions-core/hdfs-storage/src/main/java/org/apache/druid/firehose/hdfs/HdfsFirehoseFactory.java delete mode 100644 extensions-core/hdfs-storage/src/test/java/org/apache/druid/firehose/hdfs/HdfsFirehoseFactoryTest.java delete mode 100644 extensions-core/s3-extensions/src/main/java/org/apache/druid/firehose/s3/S3FirehoseDruidModule.java delete mode 100644 extensions-core/s3-extensions/src/main/java/org/apache/druid/firehose/s3/StaticS3FirehoseFactory.java delete mode 100644 extensions-core/s3-extensions/src/test/java/org/apache/druid/firehose/s3/StaticS3FirehoseFactoryTest.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/ReingestionTimelineUtils.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java rename indexing-service/src/test/java/org/apache/druid/indexing/common/{TestRealtimeTask.java => TestIndexTask.java} (61%) delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java delete mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningFirehoseFactoryIndexTest.java delete mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json rename integration-tests-ex/cases/src/test/resources/indexer/{wikipedia_combining_firehose_index_queries.json => wikipedia_combining_input_source_index_queries.json} (100%) delete mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_transform.json delete mode 100644 integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json rename integration-tests-ex/cases/src/test/resources/multi-stage-query/{wikipedia_combining_firehose_index_queries.json => wikipedia_combining_input_source_index_queries.json} (100%) delete mode 100644 integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningFirehoseFactoryIndexTest.java delete mode 100644 integration-tests/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json rename integration-tests/src/test/resources/indexer/{wikipedia_combining_firehose_index_queries.json => wikipedia_combining_input_source_index_queries.json} (100%) delete mode 100644 integration-tests/src/test/resources/indexer/wikipedia_index_task_with_transform.json delete mode 100644 integration-tests/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json delete mode 100644 processing/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java delete mode 100644 processing/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java delete mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java delete mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/FileIteratingFirehose.java delete mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java delete mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/prefetch/CacheManager.java delete mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/prefetch/FetchConfig.java delete mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/prefetch/FetchedFile.java delete mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/prefetch/Fetcher.java delete mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/prefetch/FileFetcher.java delete mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/prefetch/OpenObject.java delete mode 100644 processing/src/main/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactory.java delete mode 100644 processing/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java delete mode 100644 processing/src/test/java/org/apache/druid/data/input/impl/FileIteratingFirehoseTest.java delete mode 100644 processing/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java delete mode 100644 processing/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/firehose/CombiningFirehoseFactory.java delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactory.java delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehose.java delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseFactory.java delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactory.java delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/firehose/PrefetchSqlFirehoseFactory.java delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFetcher.java delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehose.java delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactory.java delete mode 100644 server/src/test/java/org/apache/druid/segment/realtime/firehose/CombiningFirehoseFactoryTest.java delete mode 100644 server/src/test/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactoryTest.java delete mode 100644 server/src/test/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseFactoryTest.java delete mode 100644 server/src/test/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseTest.java delete mode 100644 server/src/test/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java delete mode 100644 server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java delete mode 100644 server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseTest.java diff --git a/.github/workflows/reusable-revised-its.yml b/.github/workflows/reusable-revised-its.yml index dc3482bb5f5..8d0b57abf04 100644 --- a/.github/workflows/reusable-revised-its.yml +++ b/.github/workflows/reusable-revised-its.yml @@ -95,7 +95,7 @@ jobs: source ./integration-tests-ex/image/target/env.sh docker tag $DRUID_IT_IMAGE_NAME $DRUID_IT_IMAGE_NAME-jdk${{ matrix.jdk }} echo $DRUID_IT_IMAGE_NAME - docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ matrix.jdk }}.tar.gz + docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ inputs.build_jdk }}.tar.gz - name: Load docker image run: | diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 10c95f9022c..5fbf4b0c07d 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -576,7 +576,7 @@ This deep storage is used to interface with Cassandra. Note that the `druid-cas #### HDFS input source You can set the following property to specify permissible protocols for -the [HDFS input source](../ingestion/native-batch-input-source.md#hdfs-input-source) and the [HDFS firehose](../ingestion/native-batch-firehose.md#hdfsfirehose). +the [HDFS input source](../ingestion/native-batch-input-source.md#hdfs-input-source). |Property|Possible Values|Description|Default| |--------|---------------|-----------|-------| @@ -586,7 +586,7 @@ the [HDFS input source](../ingestion/native-batch-input-source.md#hdfs-input-sou #### HTTP input source You can set the following property to specify permissible protocols for -the [HTTP input source](../ingestion/native-batch-input-source.md#http-input-source) and the [HTTP firehose](../ingestion/native-batch-firehose.md#httpfirehose). +the [HTTP input source](../ingestion/native-batch-input-source.md#http-input-source). |Property|Possible Values|Description|Default| |--------|---------------|-----------|-------| @@ -599,7 +599,6 @@ the [HTTP input source](../ingestion/native-batch-input-source.md#http-input-sou You can use the following properties to specify permissible JDBC options for: - [SQL input source](../ingestion/native-batch-input-source.md#sql-input-source) -- [SQL firehose](../ingestion/native-batch-firehose.md#sqlfirehose), - [globally cached JDBC lookups](../development/extensions-core/lookups-cached-global.md#jdbc-lookup) - [JDBC Data Fetcher for per-lookup caching](../development/extensions-core/druid-lookups.md#data-fetcher-layer). diff --git a/docs/development/extensions-core/google.md b/docs/development/extensions-core/google.md index eaed4eb567c..813f9827e90 100644 --- a/docs/development/extensions-core/google.md +++ b/docs/development/extensions-core/google.md @@ -40,8 +40,6 @@ The [Google Cloud Storage input source](../../ingestion/native-batch-input-sourc to read objects directly from Google Cloud Storage. If you use the [Hadoop task](../../ingestion/hadoop.md), you can read data from Google Cloud Storage by specifying the paths in your [`inputSpec`](../../ingestion/hadoop.md#inputspec). -Objects can also be read directly from Google Cloud Storage via the [StaticGoogleBlobStoreFirehose](../../ingestion/native-batch-firehose.md#staticgoogleblobstorefirehose) - ### Deep Storage Deep storage can be written to Google Cloud Storage either via this extension or the [druid-hdfs-storage extension](../extensions-core/hdfs.md). diff --git a/docs/development/extensions-core/mysql.md b/docs/development/extensions-core/mysql.md index 7cb013cbdaf..f7c300c16ac 100644 --- a/docs/development/extensions-core/mysql.md +++ b/docs/development/extensions-core/mysql.md @@ -125,7 +125,7 @@ If using the MariaDB connector library, set `druid.metadata.mysql.driver.driverC |`druid.metadata.mysql.ssl.enabledSSLCipherSuites`|Overrides the existing cipher suites with these cipher suites.|none|no| |`druid.metadata.mysql.ssl.enabledTLSProtocols`|Overrides the TLS protocols with these protocols.|none|no| -### MySQL InputSource and Firehose +### MySQL InputSource ```json { @@ -133,22 +133,17 @@ If using the MariaDB connector library, set `druid.metadata.mysql.driver.driverC "spec": { "dataSchema": { "dataSource": "some_datasource", - "parser": { - "parseSpec": { - "format": "timeAndDims", - "dimensionsSpec": { - "dimensionExclusions": [], - "dimensions": [ - "dim1", - "dim2", - "dim3" - ] - }, - "timestampSpec": { - "format": "auto", - "column": "ts" - } - } + "dimensionsSpec": { + "dimensionExclusions": [], + "dimensions": [ + "dim1", + "dim2", + "dim3" + ] + }, + "timestampSpec": { + "format": "auto", + "column": "ts" }, "metricsSpec": [], "granularitySpec": { @@ -167,7 +162,7 @@ If using the MariaDB connector library, set `druid.metadata.mysql.driver.driverC }, "ioConfig": { "type": "index_parallel", - "firehose": { + "inputSource": { "type": "sql", "database": { "type": "mysql", @@ -180,6 +175,9 @@ If using the MariaDB connector library, set `druid.metadata.mysql.driver.driverC "sqls": [ "SELECT * FROM some_table" ] + }, + "inputFormat": { + "type": "json" } }, "tuningConfig": { diff --git a/docs/development/extensions-core/postgresql.md b/docs/development/extensions-core/postgresql.md index 8fd2ece0d96..07e17d1f292 100644 --- a/docs/development/extensions-core/postgresql.md +++ b/docs/development/extensions-core/postgresql.md @@ -87,7 +87,7 @@ In most cases, the configuration options map directly to the [postgres JDBC conn ### PostgreSQL Firehose -The PostgreSQL extension provides an implementation of an [SqlFirehose](../../ingestion/native-batch-firehose.md) which can be used to ingest data into Druid from a PostgreSQL database. +The PostgreSQL extension provides an implementation of an [SQL input source](../../ingestion/native-batch-input-source.md) which can be used to ingest data into Druid from a PostgreSQL database. ```json { @@ -95,22 +95,17 @@ The PostgreSQL extension provides an implementation of an [SqlFirehose](../../in "spec": { "dataSchema": { "dataSource": "some_datasource", - "parser": { - "parseSpec": { - "format": "timeAndDims", - "dimensionsSpec": { - "dimensionExclusions": [], - "dimensions": [ - "dim1", - "dim2", - "dim3" - ] - }, - "timestampSpec": { - "format": "auto", - "column": "ts" - } - } + "dimensionsSpec": { + "dimensionExclusions": [], + "dimensions": [ + "dim1", + "dim2", + "dim3" + ] + }, + "timestampSpec": { + "format": "auto", + "column": "ts" }, "metricsSpec": [], "granularitySpec": { @@ -129,7 +124,7 @@ The PostgreSQL extension provides an implementation of an [SqlFirehose](../../in }, "ioConfig": { "type": "index_parallel", - "firehose": { + "inputSource": { "type": "sql", "database": { "type": "postgresql", @@ -142,6 +137,9 @@ The PostgreSQL extension provides an implementation of an [SqlFirehose](../../in "sqls": [ "SELECT * FROM some_table" ] + }, + "inputFormat": { + "type": "json" } }, "tuningConfig": { diff --git a/docs/ingestion/native-batch.md b/docs/ingestion/native-batch.md index e8d83f34046..cd2ef01e017 100644 --- a/docs/ingestion/native-batch.md +++ b/docs/ingestion/native-batch.md @@ -263,7 +263,7 @@ The size-based split hint spec affects all splittable input sources except for t #### Segments Split Hint Spec -The segments split hint spec is used only for [`DruidInputSource`](./native-batch-input-source.md) and legacy `IngestSegmentFirehose`. +The segments split hint spec is used only for [`DruidInputSource`](./native-batch-input-source.md). |property|description|default|required?| |--------|-----------|-------|---------| @@ -588,7 +588,7 @@ An example of the result is "filter": "lineitem.tbl.5" }, "inputFormat": { - "format": "tsv", + "type": "tsv", "delimiter": "|", "columns": [ "l_orderkey", diff --git a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/firehose/aliyun/OssFirehoseDruidModule.java b/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/firehose/aliyun/OssFirehoseDruidModule.java deleted file mode 100644 index 864717657fe..00000000000 --- a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/firehose/aliyun/OssFirehoseDruidModule.java +++ /dev/null @@ -1,46 +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.druid.firehose.aliyun; - -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.jsontype.NamedType; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.google.common.collect.ImmutableList; -import com.google.inject.Binder; -import org.apache.druid.initialization.DruidModule; - -import java.util.List; - -public class OssFirehoseDruidModule implements DruidModule -{ - @Override - public List getJacksonModules() - { - return ImmutableList.of( - new SimpleModule().registerSubtypes(new NamedType(StaticOssFirehoseFactory.class, "static-aliyun-oss")) - ); - } - - @Override - public void configure(Binder binder) - { - - } -} diff --git a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/firehose/aliyun/StaticOssFirehoseFactory.java b/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/firehose/aliyun/StaticOssFirehoseFactory.java deleted file mode 100644 index d71198826dc..00000000000 --- a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/firehose/aliyun/StaticOssFirehoseFactory.java +++ /dev/null @@ -1,243 +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.druid.firehose.aliyun; - -import com.aliyun.oss.OSS; -import com.aliyun.oss.OSSException; -import com.aliyun.oss.model.GetObjectRequest; -import com.aliyun.oss.model.OSSObject; -import com.aliyun.oss.model.OSSObjectSummary; -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.storage.aliyun.OssStorageDruidModule; -import org.apache.druid.storage.aliyun.OssUtils; -import org.apache.druid.utils.CompressionUtils; - -import java.io.IOException; -import java.io.InputStream; -import java.net.URI; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; - -/** - * Builds firehoses that read from a predefined list of aliyun OSS objects and then dry up. - */ -public class StaticOssFirehoseFactory extends PrefetchableTextFilesFirehoseFactory -{ - private static final Logger log = new Logger(StaticOssFirehoseFactory.class); - - private final OSS client; - private final List uris; - private final List prefixes; - - @JsonCreator - public StaticOssFirehoseFactory( - @JacksonInject OSS client, - @JsonProperty("uris") List uris, - @JsonProperty("prefixes") List prefixes, - @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, - @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, - @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, - @JsonProperty("fetchTimeout") Long fetchTimeout, - @JsonProperty("maxFetchRetry") Integer maxFetchRetry - ) - { - super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); - this.client = Preconditions.checkNotNull(client, "client"); - this.uris = uris == null ? new ArrayList<>() : uris; - this.prefixes = prefixes == null ? new ArrayList<>() : prefixes; - - if (!this.uris.isEmpty() && !this.prefixes.isEmpty()) { - throw new IAE("uris and prefixes cannot be used together"); - } - - if (this.uris.isEmpty() && this.prefixes.isEmpty()) { - throw new IAE("uris or prefixes must be specified"); - } - - for (final URI inputURI : this.uris) { - Preconditions.checkArgument(OssStorageDruidModule.SCHEME.equals(inputURI.getScheme()), - "input uri scheme == %s (%s)", - OssStorageDruidModule.SCHEME, - inputURI); - } - - for (final URI inputURI : this.prefixes) { - Preconditions.checkArgument(OssStorageDruidModule.SCHEME.equals(inputURI.getScheme()), - "input uri scheme == %s (%s)", - OssStorageDruidModule.SCHEME, - inputURI); - } - } - - @JsonProperty - public List getUris() - { - return uris; - } - - @JsonProperty("prefixes") - public List getPrefixes() - { - return prefixes; - } - - @Override - protected Collection initObjects() - { - if (!uris.isEmpty()) { - return uris; - } else { - final List objects = new ArrayList<>(); - for (final URI prefix : prefixes) { - final Iterator objectSummaryIterator = OssUtils.objectSummaryIterator( - client, - Collections.singletonList(prefix), - OssUtils.MAX_LISTING_LENGTH - ); - - objectSummaryIterator.forEachRemaining(objects::add); - } - return objects.stream().map(OssUtils::summaryToUri).collect(Collectors.toList()); - } - } - - @Override - protected InputStream openObjectStream(URI object) throws IOException - { - try { - // Get data of the given object and open an input stream - final String bucket = object.getAuthority(); - final String key = OssUtils.extractKey(object); - - final OSSObject ossObject = client.getObject(bucket, key); - if (ossObject == null) { - throw new ISE("Failed to get an Aliyun OSS object for bucket[%s] and key[%s]", bucket, key); - } - return ossObject.getObjectContent(); - } - catch (OSSException e) { - throw new IOException(e); - } - } - - @Override - protected InputStream openObjectStream(URI object, long start) throws IOException - { - final String bucket = object.getAuthority(); - final String key = OssUtils.extractKey(object); - - final GetObjectRequest request = new GetObjectRequest(bucket, key); - try { - final OSSObject ossObject = client.getObject(request); - if (ossObject == null) { - throw new ISE( - "Failed to get an Aliyun OSS object for bucket[%s], key[%s], and start[%d]", - bucket, - key, - start - ); - } - InputStream is = ossObject.getObjectContent(); - is.skip(start); - return is; - } - catch (OSSException e) { - throw new IOException(e); - } - } - - @Override - protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException - { - return CompressionUtils.decompress(stream, OssUtils.extractKey(object)); - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - StaticOssFirehoseFactory that = (StaticOssFirehoseFactory) o; - - return Objects.equals(uris, that.uris) && - Objects.equals(prefixes, that.prefixes) && - getMaxCacheCapacityBytes() == that.getMaxCacheCapacityBytes() && - getMaxFetchCapacityBytes() == that.getMaxFetchCapacityBytes() && - getPrefetchTriggerBytes() == that.getPrefetchTriggerBytes() && - getFetchTimeout() == that.getFetchTimeout() && - getMaxFetchRetry() == that.getMaxFetchRetry(); - } - - @Override - public int hashCode() - { - return Objects.hash( - uris, - prefixes, - getMaxCacheCapacityBytes(), - getMaxFetchCapacityBytes(), - getPrefetchTriggerBytes(), - getFetchTimeout(), - getMaxFetchRetry() - ); - } - - @Override - protected Predicate getRetryCondition() - { - return OssUtils.RETRYABLE; - } - - @Override - public FiniteFirehoseFactory withSplit(InputSplit split) - { - return new StaticOssFirehoseFactory( - client, - Collections.singletonList(split.get()), - null, - getMaxCacheCapacityBytes(), - getMaxFetchCapacityBytes(), - getPrefetchTriggerBytes(), - getFetchTimeout(), - getMaxFetchRetry() - ); - } -} diff --git a/extensions-contrib/aliyun-oss-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/aliyun-oss-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule index 3d434e7c902..8613c838013 100644 --- a/extensions-contrib/aliyun-oss-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule +++ b/extensions-contrib/aliyun-oss-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -14,5 +14,4 @@ # limitations under the License. org.apache.druid.storage.aliyun.OssStorageDruidModule -org.apache.druid.firehose.aliyun.OssFirehoseDruidModule org.apache.druid.data.input.aliyun.OssInputSourceDruidModule \ No newline at end of file diff --git a/extensions-contrib/cloudfiles-extensions/pom.xml b/extensions-contrib/cloudfiles-extensions/pom.xml index 4f90c077267..e4860d98b9c 100644 --- a/extensions-contrib/cloudfiles-extensions/pom.xml +++ b/extensions-contrib/cloudfiles-extensions/pom.xml @@ -143,10 +143,6 @@ rackspace-cloudfiles-uk ${jclouds.version} - - com.fasterxml.jackson.module - jackson-module-guice - diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesBlob.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesBlob.java deleted file mode 100644 index edc0bbb65c0..00000000000 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesBlob.java +++ /dev/null @@ -1,101 +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.druid.firehose.cloudfiles; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import javax.validation.constraints.NotNull; -import java.util.Objects; - -public class CloudFilesBlob -{ - @JsonProperty - @NotNull - private String container; - - @JsonProperty - @NotNull - private String path; - - @JsonProperty - @NotNull - private String region; - - @JsonCreator - public CloudFilesBlob( - @JsonProperty("container") String container, - @JsonProperty("path") String path, - @JsonProperty("region") String region - ) - { - this.container = container; - this.path = path; - this.region = region; - } - - public String getContainer() - { - return container; - } - - public String getPath() - { - return path; - } - - public String getRegion() - { - return region; - } - - @Override - public String toString() - { - return "CloudFilesBlob{" - + "container=" + container - + ",path=" + path - + ",region=" + region - + "}"; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - final CloudFilesBlob that = (CloudFilesBlob) o; - return Objects.equals(container, that.container) && - Objects.equals(path, that.path) && - Objects.equals(region, that.region); - } - - @Override - public int hashCode() - { - return Objects.hash(container, path, region); - } -} diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesFirehoseDruidModule.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesFirehoseDruidModule.java deleted file mode 100644 index 3531e7584e4..00000000000 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesFirehoseDruidModule.java +++ /dev/null @@ -1,48 +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.druid.firehose.cloudfiles; - -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.jsontype.NamedType; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.google.common.collect.ImmutableList; -import com.google.inject.Binder; -import org.apache.druid.initialization.DruidModule; - -import java.util.List; - -public class CloudFilesFirehoseDruidModule implements DruidModule -{ - - @Override - public List getJacksonModules() - { - return ImmutableList.of( - new SimpleModule().registerSubtypes( - new NamedType(StaticCloudFilesFirehoseFactory.class, "staticcloudfiles"))); - } - - @Override - public void configure(Binder arg0) - { - - } - -} diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java deleted file mode 100644 index f0de9f7e98d..00000000000 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java +++ /dev/null @@ -1,162 +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.druid.firehose.cloudfiles; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Predicate; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.storage.cloudfiles.CloudFilesByteSource; -import org.apache.druid.storage.cloudfiles.CloudFilesObjectApiProxy; -import org.apache.druid.storage.cloudfiles.CloudFilesUtils; -import org.apache.druid.utils.CompressionUtils; -import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi; - -import java.io.IOException; -import java.io.InputStream; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Objects; - -public class StaticCloudFilesFirehoseFactory extends PrefetchableTextFilesFirehoseFactory -{ - private static final Logger log = new Logger(StaticCloudFilesFirehoseFactory.class); - - private final CloudFilesApi cloudFilesApi; - private final List blobs; - - @JsonCreator - public StaticCloudFilesFirehoseFactory( - @JacksonInject CloudFilesApi cloudFilesApi, - @JsonProperty("blobs") List blobs, - @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, - @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, - @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, - @JsonProperty("fetchTimeout") Long fetchTimeout, - @JsonProperty("maxFetchRetry") Integer maxFetchRetry - ) - { - super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); - this.cloudFilesApi = cloudFilesApi; - this.blobs = blobs; - } - - @JsonProperty - public List getBlobs() - { - return blobs; - } - - @Override - protected Collection initObjects() - { - return blobs; - } - - @Override - protected InputStream openObjectStream(CloudFilesBlob object) throws IOException - { - return openObjectStream(object, 0); - } - - @Override - protected InputStream openObjectStream(CloudFilesBlob object, long start) throws IOException - { - return createCloudFilesByteSource(object).openStream(start); - } - - private CloudFilesByteSource createCloudFilesByteSource(CloudFilesBlob object) - { - final String region = object.getRegion(); - final String container = object.getContainer(); - final String path = object.getPath(); - - log.info("Retrieving file from region[%s], container[%s] and path [%s]", - region, container, path - ); - CloudFilesObjectApiProxy objectApi = new CloudFilesObjectApiProxy(cloudFilesApi, region, container); - return new CloudFilesByteSource(objectApi, path); - } - - @Override - protected InputStream wrapObjectStream(CloudFilesBlob object, InputStream stream) throws IOException - { - return CompressionUtils.decompress(stream, object.getPath()); - } - - @Override - public boolean equals(Object o) - { - if (o == this) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - final StaticCloudFilesFirehoseFactory that = (StaticCloudFilesFirehoseFactory) o; - return Objects.equals(blobs, that.blobs) && - getMaxCacheCapacityBytes() == that.getMaxCacheCapacityBytes() && - getMaxFetchCapacityBytes() == that.getMaxFetchCapacityBytes() && - getPrefetchTriggerBytes() == that.getPrefetchTriggerBytes() && - getFetchTimeout() == that.getFetchTimeout() && - getMaxFetchRetry() == that.getMaxFetchRetry(); - } - - @Override - public int hashCode() - { - return Objects.hash( - blobs, - getMaxCacheCapacityBytes(), - getMaxFetchCapacityBytes(), - getPrefetchTriggerBytes(), - getFetchTimeout(), - getMaxFetchRetry() - ); - } - - @Override - protected Predicate getRetryCondition() - { - return CloudFilesUtils.CLOUDFILESRETRY; - } - - @Override - public FiniteFirehoseFactory withSplit(InputSplit split) - { - return new StaticCloudFilesFirehoseFactory( - cloudFilesApi, - Collections.singletonList(split.get()), - getMaxCacheCapacityBytes(), - getMaxFetchCapacityBytes(), - getPrefetchTriggerBytes(), - getFetchTimeout(), - getMaxFetchRetry() - ); - } -} diff --git a/extensions-contrib/cloudfiles-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/cloudfiles-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule index 81a1411f89b..367f66497ba 100644 --- a/extensions-contrib/cloudfiles-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule +++ b/extensions-contrib/cloudfiles-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -16,4 +16,3 @@ # under the License. org.apache.druid.storage.cloudfiles.CloudFilesStorageDruidModule -org.apache.druid.firehose.cloudfiles.CloudFilesFirehoseDruidModule diff --git a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactoryTest.java b/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactoryTest.java deleted file mode 100644 index 48f3ca694fa..00000000000 --- a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactoryTest.java +++ /dev/null @@ -1,105 +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.druid.firehose.cloudfiles; - -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.fasterxml.jackson.module.guice.ObjectMapperModule; -import com.google.common.collect.ImmutableList; -import com.google.inject.Binder; -import com.google.inject.Guice; -import com.google.inject.Injector; -import com.google.inject.Provides; -import org.apache.druid.initialization.DruidModule; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.easymock.EasyMock; -import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi; -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; -import java.util.List; - -public class StaticCloudFilesFirehoseFactoryTest -{ - private static final CloudFilesApi API = EasyMock.niceMock(CloudFilesApi.class); - - @Test - public void testSerde() throws IOException - { - final ObjectMapper mapper = createObjectMapper(new TestModule()); - - final List blobs = ImmutableList.of( - new CloudFilesBlob("container", "foo", "bar"), - new CloudFilesBlob("container", "foo", "bar2") - ); - - final StaticCloudFilesFirehoseFactory factory = new StaticCloudFilesFirehoseFactory( - API, - blobs, - 2048L, - 1024L, - 512L, - 100L, - 5 - ); - - final StaticCloudFilesFirehoseFactory outputFact = mapper.readValue( - mapper.writeValueAsString(factory), - StaticCloudFilesFirehoseFactory.class - ); - - Assert.assertEquals(factory, outputFact); - } - - private static ObjectMapper createObjectMapper(DruidModule baseModule) - { - final ObjectMapper baseMapper = new DefaultObjectMapper(); - baseModule.getJacksonModules().forEach(baseMapper::registerModule); - - final Injector injector = Guice.createInjector( - new ObjectMapperModule(), - baseModule - ); - return injector.getInstance(ObjectMapper.class); - } - - private static class TestModule implements DruidModule - { - @Override - public List getJacksonModules() - { - return ImmutableList.of(new SimpleModule()); - } - - @Override - public void configure(Binder binder) - { - - } - - @Provides - public CloudFilesApi getRestS3Service() - { - return API; - } - } -} diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/firehose/azure/AzureBlob.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/firehose/azure/AzureBlob.java deleted file mode 100644 index ef8278eff91..00000000000 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/firehose/azure/AzureBlob.java +++ /dev/null @@ -1,93 +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.druid.firehose.azure; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import javax.validation.constraints.NotNull; -import java.util.Objects; - -/** - * Represents an Azure based blob. Used with {@link StaticAzureBlobStoreFirehoseFactory}. - * - * @deprecated as of version 0.18.0 because support for firehose has been discontinued. Please use - * {@link org.apache.druid.data.input.azure.AzureEntity} with {@link org.apache.druid.data.input.azure.AzureInputSource} - * instead. - */ -@Deprecated -public class AzureBlob -{ - @JsonProperty - @NotNull - private String container; - - @JsonProperty - @NotNull - private String path; - - @JsonCreator - public AzureBlob(@JsonProperty("container") String container, @JsonProperty("path") String path) - { - this.container = container; - this.path = path; - } - - public String getContainer() - { - return container; - } - - public String getPath() - { - return path; - } - - @Override - public String toString() - { - return "AzureBlob{" - + "container=" + container - + ",path=" + path - + "}"; - } - - @Override - public boolean equals(Object o) - { - if (o == this) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - final AzureBlob that = (AzureBlob) o; - return Objects.equals(container, that.container) && - Objects.equals(path, that.path); - } - - @Override - public int hashCode() - { - return Objects.hash(container, path); - } -} diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java deleted file mode 100644 index 449873523f2..00000000000 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java +++ /dev/null @@ -1,167 +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.druid.firehose.azure; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.storage.azure.AzureByteSource; -import org.apache.druid.storage.azure.AzureStorage; -import org.apache.druid.storage.azure.AzureUtils; -import org.apache.druid.utils.CompressionUtils; - -import java.io.IOException; -import java.io.InputStream; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Objects; - -/** - * This class is heavily inspired by the StaticS3FirehoseFactory class in the org.apache.druid.firehose.s3 package - * - * @deprecated as of version 0.18.0 because support for firehose has been discontinued. Please use - * {@link org.apache.druid.data.input.azure.AzureInputSource} instead. - */ -@Deprecated -public class StaticAzureBlobStoreFirehoseFactory extends PrefetchableTextFilesFirehoseFactory -{ - private final AzureStorage azureStorage; - private final List blobs; - - @JsonCreator - public StaticAzureBlobStoreFirehoseFactory( - @JacksonInject AzureStorage azureStorage, - @JsonProperty("blobs") List blobs, - @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, - @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, - @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, - @JsonProperty("fetchTimeout") Long fetchTimeout, - @JsonProperty("maxFetchRetry") Integer maxFetchRetry - ) - { - super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); - this.blobs = blobs; - this.azureStorage = azureStorage; - } - - @JsonProperty - public List getBlobs() - { - return blobs; - } - - @Override - protected Collection initObjects() - { - return blobs; - } - - @Override - protected InputStream openObjectStream(AzureBlob object) throws IOException - { - return makeByteSource(azureStorage, object).openStream(); - } - - @Override - protected InputStream openObjectStream(AzureBlob object, long start) throws IOException - { - // BlobInputStream.skip() moves the next read offset instead of skipping first 'start' bytes. - final InputStream in = openObjectStream(object); - final long skip = in.skip(start); - Preconditions.checkState(skip == start, "start offset was [%s] but [%s] bytes were skipped", start, skip); - return in; - } - - @Override - protected InputStream wrapObjectStream(AzureBlob object, InputStream stream) throws IOException - { - return CompressionUtils.decompress(stream, object.getPath()); - } - - private static AzureByteSource makeByteSource(AzureStorage azureStorage, AzureBlob object) - { - final String container = object.getContainer(); - final String path = StringUtils.maybeRemoveLeadingSlash(object.getPath()); - - return new AzureByteSource(azureStorage, container, path); - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - final StaticAzureBlobStoreFirehoseFactory that = (StaticAzureBlobStoreFirehoseFactory) o; - - return Objects.equals(blobs, that.blobs) && - getMaxCacheCapacityBytes() == that.getMaxCacheCapacityBytes() && - getMaxFetchCapacityBytes() == that.getMaxFetchCapacityBytes() && - getPrefetchTriggerBytes() == that.getPrefetchTriggerBytes() && - getFetchTimeout() == that.getFetchTimeout() && - getMaxFetchRetry() == that.getMaxFetchRetry(); - } - - @Override - public int hashCode() - { - return Objects.hash( - blobs, - getMaxCacheCapacityBytes(), - getMaxFetchCapacityBytes(), - getPrefetchTriggerBytes(), - getFetchTimeout(), - getMaxFetchRetry() - ); - } - - @Override - protected Predicate getRetryCondition() - { - return AzureUtils.AZURE_RETRY; - } - - @Override - public FiniteFirehoseFactory withSplit(InputSplit split) - { - return new StaticAzureBlobStoreFirehoseFactory( - azureStorage, - Collections.singletonList(split.get()), - getMaxCacheCapacityBytes(), - getMaxFetchCapacityBytes(), - getPrefetchTriggerBytes(), - getFetchTimeout(), - getMaxFetchRetry() - ); - } -} diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorageDruidModule.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorageDruidModule.java index e870aa08eca..c7c4655a3d2 100644 --- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorageDruidModule.java +++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureStorageDruidModule.java @@ -33,7 +33,6 @@ import com.microsoft.azure.storage.CloudStorageAccount; import com.microsoft.azure.storage.blob.CloudBlobClient; import org.apache.druid.data.input.azure.AzureEntityFactory; import org.apache.druid.data.input.azure.AzureInputSource; -import org.apache.druid.firehose.azure.StaticAzureBlobStoreFirehoseFactory; import org.apache.druid.guice.Binders; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; @@ -84,7 +83,6 @@ public class AzureStorageDruidModule implements DruidModule } }, new SimpleModule().registerSubtypes( - new NamedType(StaticAzureBlobStoreFirehoseFactory.class, "static-azure-blobstore"), new NamedType(AzureInputSource.class, SCHEME) ) ); diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactoryTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactoryTest.java deleted file mode 100644 index 52751e21171..00000000000 --- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactoryTest.java +++ /dev/null @@ -1,104 +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.druid.firehose.azure; - -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.fasterxml.jackson.module.guice.ObjectMapperModule; -import com.google.common.collect.ImmutableList; -import com.google.inject.Binder; -import com.google.inject.Guice; -import com.google.inject.Injector; -import com.google.inject.Provides; -import org.apache.druid.initialization.DruidModule; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.storage.azure.AzureStorage; -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; -import java.util.List; - -public class StaticAzureBlobStoreFirehoseFactoryTest -{ - private static final AzureStorage STORAGE = new AzureStorage(null); - - @Test - public void testSerde() throws IOException - { - final ObjectMapper mapper = createObjectMapper(new TestModule()); - - final List blobs = ImmutableList.of( - new AzureBlob("foo", "bar"), - new AzureBlob("foo", "bar2") - ); - - final StaticAzureBlobStoreFirehoseFactory factory = new StaticAzureBlobStoreFirehoseFactory( - STORAGE, - blobs, - 2048L, - 1024L, - 512L, - 100L, - 5 - ); - - final StaticAzureBlobStoreFirehoseFactory outputFact = mapper.readValue( - mapper.writeValueAsString(factory), - StaticAzureBlobStoreFirehoseFactory.class - ); - - Assert.assertEquals(factory, outputFact); - } - - private static ObjectMapper createObjectMapper(DruidModule baseModule) - { - final ObjectMapper baseMapper = new DefaultObjectMapper(); - baseModule.getJacksonModules().forEach(baseMapper::registerModule); - - final Injector injector = Guice.createInjector( - new ObjectMapperModule(), - baseModule - ); - return injector.getInstance(ObjectMapper.class); - } - - private static class TestModule implements DruidModule - { - @Override - public List getJacksonModules() - { - return ImmutableList.of(new SimpleModule()); - } - - @Override - public void configure(Binder binder) - { - - } - - @Provides - public AzureStorage getRestS3Service() - { - return STORAGE; - } - } -} diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/firehose/google/GoogleBlob.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/firehose/google/GoogleBlob.java deleted file mode 100644 index 0e51b62080c..00000000000 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/firehose/google/GoogleBlob.java +++ /dev/null @@ -1,81 +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.druid.firehose.google; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.util.Objects; - -public class GoogleBlob -{ - private final String bucket; - private final String path; - - @JsonCreator - public GoogleBlob(@JsonProperty("bucket") String bucket, @JsonProperty("path") String path) - { - this.bucket = bucket; - this.path = path; - } - - @JsonProperty - public String getBucket() - { - return bucket; - } - - @JsonProperty - public String getPath() - { - return path; - } - - @Override - public String toString() - { - return "GoogleBlob {" - + "bucket=" + bucket - + ",path=" + path - + "}"; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - final GoogleBlob that = (GoogleBlob) o; - return Objects.equals(bucket, that.bucket) && - Objects.equals(path, that.path); - } - - @Override - public int hashCode() - { - return Objects.hash(bucket, path); - } -} diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java deleted file mode 100644 index 22437d780e1..00000000000 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java +++ /dev/null @@ -1,155 +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.druid.firehose.google; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Predicate; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.storage.google.GoogleByteSource; -import org.apache.druid.storage.google.GoogleStorage; -import org.apache.druid.storage.google.GoogleUtils; -import org.apache.druid.utils.CompressionUtils; - -import java.io.IOException; -import java.io.InputStream; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Objects; - -public class StaticGoogleBlobStoreFirehoseFactory extends PrefetchableTextFilesFirehoseFactory -{ - private final GoogleStorage storage; - private final List blobs; - - @JsonCreator - public StaticGoogleBlobStoreFirehoseFactory( - @JacksonInject GoogleStorage storage, - @JsonProperty("blobs") List blobs, - @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, - @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, - @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, - @JsonProperty("fetchTimeout") Long fetchTimeout, - @JsonProperty("maxFetchRetry") Integer maxFetchRetry - ) - { - super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); - this.storage = storage; - this.blobs = blobs; - } - - @JsonProperty - public List getBlobs() - { - return blobs; - } - - @Override - protected Collection initObjects() - { - return blobs; - } - - @Override - protected InputStream openObjectStream(GoogleBlob object) throws IOException - { - return openObjectStream(object, 0); - } - - @Override - protected InputStream openObjectStream(GoogleBlob object, long start) throws IOException - { - return createGoogleByteSource(object).openStream(start); - } - - private GoogleByteSource createGoogleByteSource(GoogleBlob object) - { - final String bucket = object.getBucket(); - final String path = StringUtils.maybeRemoveLeadingSlash(object.getPath()); - - return new GoogleByteSource(storage, bucket, path); - } - - @Override - protected InputStream wrapObjectStream(GoogleBlob object, InputStream stream) throws IOException - { - return CompressionUtils.decompress(stream, object.getPath()); - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - final StaticGoogleBlobStoreFirehoseFactory that = (StaticGoogleBlobStoreFirehoseFactory) o; - - return Objects.equals(blobs, that.blobs) && - getMaxCacheCapacityBytes() == that.getMaxCacheCapacityBytes() && - getMaxFetchCapacityBytes() == that.getMaxFetchCapacityBytes() && - getPrefetchTriggerBytes() == that.getPrefetchTriggerBytes() && - getFetchTimeout() == that.getFetchTimeout() && - getMaxFetchRetry() == that.getMaxFetchRetry(); - } - - @Override - public int hashCode() - { - return Objects.hash( - blobs, - getMaxCacheCapacityBytes(), - getMaxFetchCapacityBytes(), - getPrefetchTriggerBytes(), - getFetchTimeout(), - getMaxFetchRetry() - ); - } - - @Override - protected Predicate getRetryCondition() - { - return GoogleUtils::isRetryable; - } - - @Override - public FiniteFirehoseFactory withSplit(InputSplit split) - { - return new StaticGoogleBlobStoreFirehoseFactory( - storage, - Collections.singletonList(split.get()), - getMaxCacheCapacityBytes(), - getMaxFetchCapacityBytes(), - getPrefetchTriggerBytes(), - getFetchTimeout(), - getMaxFetchRetry() - ); - } -} - diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorageDruidModule.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorageDruidModule.java index 838b46f2287..8aa77b0298f 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorageDruidModule.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/GoogleStorageDruidModule.java @@ -34,7 +34,6 @@ import com.google.inject.Provides; import com.google.inject.multibindings.MapBinder; import org.apache.druid.data.SearchableVersionedDataFinder; import org.apache.druid.data.input.google.GoogleCloudStorageInputSource; -import org.apache.druid.firehose.google.StaticGoogleBlobStoreFirehoseFactory; import org.apache.druid.guice.Binders; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; @@ -77,7 +76,6 @@ public class GoogleStorageDruidModule implements DruidModule } }, new SimpleModule().registerSubtypes( - new NamedType(StaticGoogleBlobStoreFirehoseFactory.class, "static-google-blobstore"), new NamedType(GoogleCloudStorageInputSource.class, SCHEME) ) ); diff --git a/extensions-core/google-extensions/src/test/java/org/apache/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactoryTest.java b/extensions-core/google-extensions/src/test/java/org/apache/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactoryTest.java deleted file mode 100644 index c9996b53ccb..00000000000 --- a/extensions-core/google-extensions/src/test/java/org/apache/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactoryTest.java +++ /dev/null @@ -1,63 +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.druid.firehose.google; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; -import org.apache.druid.data.input.google.GoogleCloudStorageInputSourceTest; -import org.apache.druid.storage.google.GoogleStorage; -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; -import java.util.List; - -public class StaticGoogleBlobStoreFirehoseFactoryTest -{ - private static final GoogleStorage STORAGE = new GoogleStorage(null); - - @Test - public void testSerde() throws IOException - { - final ObjectMapper mapper = GoogleCloudStorageInputSourceTest.createGoogleObjectMapper(); - - final List blobs = ImmutableList.of( - new GoogleBlob("foo", "bar"), - new GoogleBlob("foo", "bar2") - ); - - final StaticGoogleBlobStoreFirehoseFactory factory = new StaticGoogleBlobStoreFirehoseFactory( - STORAGE, - blobs, - 2048L, - 1024L, - 512L, - 100L, - 5 - ); - - final StaticGoogleBlobStoreFirehoseFactory outputFact = mapper.readValue( - mapper.writeValueAsString(factory), - StaticGoogleBlobStoreFirehoseFactory.class - ); - - Assert.assertEquals(factory, outputFact); - } -} diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/firehose/hdfs/HdfsFirehoseFactory.java b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/firehose/hdfs/HdfsFirehoseFactory.java deleted file mode 100644 index f7fac9f3b47..00000000000 --- a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/firehose/hdfs/HdfsFirehoseFactory.java +++ /dev/null @@ -1,127 +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.druid.firehose.hdfs; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Predicate; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; -import org.apache.druid.guice.Hdfs; -import org.apache.druid.inputsource.hdfs.HdfsInputSource; -import org.apache.druid.inputsource.hdfs.HdfsInputSourceConfig; -import org.apache.druid.storage.hdfs.HdfsDataSegmentPuller; -import org.apache.druid.utils.CompressionUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.Path; - -import java.io.IOException; -import java.io.InputStream; -import java.util.Collection; -import java.util.List; - -public class HdfsFirehoseFactory extends PrefetchableTextFilesFirehoseFactory -{ - private final List inputPaths; - private final Configuration conf; - private final HdfsInputSourceConfig inputSourceConfig; - - @JsonCreator - public HdfsFirehoseFactory( - @JsonProperty("paths") Object inputPaths, - @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, - @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, - @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, - @JsonProperty("fetchTimeout") Long fetchTimeout, - @JsonProperty("maxFetchRetry") Integer maxFetchRetry, - @JacksonInject @Hdfs Configuration conf, - @JacksonInject HdfsInputSourceConfig inputSourceConfig - ) - { - super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); - this.inputPaths = HdfsInputSource.coerceInputPathsToList(inputPaths, "paths"); - this.conf = conf; - this.inputSourceConfig = inputSourceConfig; - this.inputPaths.forEach(p -> HdfsInputSource.verifyProtocol(conf, inputSourceConfig, p)); - } - - @JsonProperty("paths") - public List getInputPaths() - { - return inputPaths; - } - - @Override - protected Collection initObjects() throws IOException - { - return HdfsInputSource.getPaths(inputPaths, conf); - } - - @Override - protected InputStream openObjectStream(Path path) throws IOException - { - return path.getFileSystem(conf).open(path); - } - - @Override - protected InputStream openObjectStream(Path path, long start) throws IOException - { - final FSDataInputStream in = path.getFileSystem(conf).open(path); - in.seek(start); - return in; - } - - @Override - protected InputStream wrapObjectStream(Path path, InputStream stream) throws IOException - { - return CompressionUtils.decompress(stream, path.getName()); - } - - @Override - protected Predicate getRetryCondition() - { - return HdfsDataSegmentPuller.RETRY_PREDICATE; - } - - @Override - public boolean isSplittable() - { - return true; - } - - @Override - public FiniteFirehoseFactory withSplit(InputSplit split) - { - return new HdfsFirehoseFactory( - split.get().toString(), - getMaxCacheCapacityBytes(), - getMaxFetchCapacityBytes(), - getPrefetchTriggerBytes(), - getFetchTimeout(), - getMaxFetchRetry(), - conf, - inputSourceConfig - ); - } -} diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsStorageDruidModule.java b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsStorageDruidModule.java index 3ca8e23535e..68c7650960b 100644 --- a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsStorageDruidModule.java +++ b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsStorageDruidModule.java @@ -26,7 +26,6 @@ import com.google.inject.Binder; import com.google.inject.Inject; import com.google.inject.multibindings.MapBinder; import org.apache.druid.data.SearchableVersionedDataFinder; -import org.apache.druid.firehose.hdfs.HdfsFirehoseFactory; import org.apache.druid.guice.Binders; import org.apache.druid.guice.Hdfs; import org.apache.druid.guice.JsonConfigProvider; @@ -66,7 +65,6 @@ public class HdfsStorageDruidModule implements DruidModule return Collections.singletonList( new SimpleModule().registerSubtypes( new NamedType(HdfsLoadSpec.class, HdfsStorageDruidModule.SCHEME), - new NamedType(HdfsFirehoseFactory.class, HdfsStorageDruidModule.SCHEME), new NamedType(HdfsInputSource.class, HdfsStorageDruidModule.SCHEME) ) ); diff --git a/extensions-core/hdfs-storage/src/test/java/org/apache/druid/firehose/hdfs/HdfsFirehoseFactoryTest.java b/extensions-core/hdfs-storage/src/test/java/org/apache/druid/firehose/hdfs/HdfsFirehoseFactoryTest.java deleted file mode 100644 index e96a773c0ce..00000000000 --- a/extensions-core/hdfs-storage/src/test/java/org/apache/druid/firehose/hdfs/HdfsFirehoseFactoryTest.java +++ /dev/null @@ -1,218 +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.druid.firehose.hdfs; - -import com.fasterxml.jackson.databind.InjectableValues.Std; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableSet; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.inputsource.hdfs.HdfsInputSourceConfig; -import org.apache.druid.storage.hdfs.HdfsStorageDruidModule; -import org.apache.hadoop.conf.Configuration; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; - -import java.io.IOException; -import java.util.Collections; - -public class HdfsFirehoseFactoryTest -{ - private static final HdfsInputSourceConfig DEFAULT_INPUT_SOURCE_CONFIG = new HdfsInputSourceConfig(null); - private static final Configuration DEFAULT_CONFIGURATION = new Configuration(); - - @BeforeClass - public static void setup() - { - DEFAULT_CONFIGURATION.set("fs.default.name", "hdfs://localhost:7020"); - } - - @Rule - public ExpectedException expectedException = ExpectedException.none(); - - @Test - public void testArrayPaths() throws IOException - { - final HdfsFirehoseFactory firehoseFactory = new HdfsFirehoseFactory( - Collections.singletonList("/foo/bar"), - null, - null, - null, - null, - null, - DEFAULT_CONFIGURATION, - DEFAULT_INPUT_SOURCE_CONFIG - ); - - final ObjectMapper mapper = createMapper(); - - final HdfsFirehoseFactory firehoseFactory2 = (HdfsFirehoseFactory) - mapper.readValue(mapper.writeValueAsString(firehoseFactory), FirehoseFactory.class); - - Assert.assertEquals( - firehoseFactory.getInputPaths(), - firehoseFactory2.getInputPaths() - ); - } - - @Test - public void testStringPaths() throws IOException - { - final HdfsFirehoseFactory firehoseFactory = new HdfsFirehoseFactory( - "/foo/bar", - null, - null, - null, - null, - null, - DEFAULT_CONFIGURATION, - DEFAULT_INPUT_SOURCE_CONFIG - ); - final ObjectMapper mapper = createMapper(); - - final HdfsFirehoseFactory firehoseFactory2 = (HdfsFirehoseFactory) - mapper.readValue(mapper.writeValueAsString(firehoseFactory), FirehoseFactory.class); - - Assert.assertEquals( - firehoseFactory.getInputPaths(), - firehoseFactory2.getInputPaths() - ); - } - - @Test - public void testConstructorAllowsOnlyDefaultProtocol() - { - new HdfsFirehoseFactory( - "hdfs://localhost:7020/foo/bar", - null, - null, - null, - null, - null, - DEFAULT_CONFIGURATION, - DEFAULT_INPUT_SOURCE_CONFIG - ); - - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("Only [hdfs] protocols are allowed"); - new HdfsFirehoseFactory( - "file:/foo/bar", - null, - null, - null, - null, - null, - DEFAULT_CONFIGURATION, - DEFAULT_INPUT_SOURCE_CONFIG - ); - } - - @Test - public void testConstructorAllowsOnlyCustomProtocol() - { - final Configuration conf = new Configuration(); - conf.set("fs.ftp.impl", "org.apache.hadoop.fs.ftp.FTPFileSystem"); - new HdfsFirehoseFactory( - "ftp://localhost:21/foo/bar", - null, - null, - null, - null, - null, - DEFAULT_CONFIGURATION, - new HdfsInputSourceConfig(ImmutableSet.of("ftp")) - ); - - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("Only [druid] protocols are allowed"); - new HdfsFirehoseFactory( - "hdfs://localhost:7020/foo/bar", - null, - null, - null, - null, - null, - DEFAULT_CONFIGURATION, - new HdfsInputSourceConfig(ImmutableSet.of("druid")) - ); - } - - @Test - public void testConstructorWithDefaultHdfs() - { - new HdfsFirehoseFactory( - "/foo/bar*", - null, - null, - null, - null, - null, - DEFAULT_CONFIGURATION, - DEFAULT_INPUT_SOURCE_CONFIG - ); - - new HdfsFirehoseFactory( - "foo/bar*", - null, - null, - null, - null, - null, - DEFAULT_CONFIGURATION, - DEFAULT_INPUT_SOURCE_CONFIG - ); - - new HdfsFirehoseFactory( - "hdfs:///foo/bar*", - null, - null, - null, - null, - null, - DEFAULT_CONFIGURATION, - DEFAULT_INPUT_SOURCE_CONFIG - ); - - new HdfsFirehoseFactory( - "hdfs://localhost:10020/foo/bar*", // different hdfs - null, - null, - null, - null, - null, - DEFAULT_CONFIGURATION, - DEFAULT_INPUT_SOURCE_CONFIG - ); - } - - private static ObjectMapper createMapper() - { - final ObjectMapper mapper = new ObjectMapper(); - new HdfsStorageDruidModule().getJacksonModules().forEach(mapper::registerModule); - mapper.setInjectableValues( - new Std() - .addValue(Configuration.class, DEFAULT_CONFIGURATION) - .addValue(HdfsInputSourceConfig.class, DEFAULT_INPUT_SOURCE_CONFIG) - ); - return mapper; - } -} diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java index abe25b53913..47327dbeee1 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java @@ -24,11 +24,15 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.curator.test.TestingCluster; import org.apache.druid.client.indexing.SamplerResponse; +import org.apache.druid.client.indexing.SamplerSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FloatDimensionSchema; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; @@ -37,6 +41,7 @@ import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; import org.apache.druid.indexing.overlord.sampler.SamplerConfig; import org.apache.druid.indexing.overlord.sampler.SamplerException; import org.apache.druid.indexing.overlord.sampler.SamplerTestUtils; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.parsers.JSONPathSpec; @@ -56,9 +61,11 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import java.io.IOException; import java.util.Arrays; import java.util.Iterator; import java.util.List; +import java.util.Map; public class KafkaSamplerSpecTest extends InitializedNullHandlingTest { @@ -167,6 +174,87 @@ public class KafkaSamplerSpecTest extends InitializedNullHandlingTest OBJECT_MAPPER ); + runSamplerAndCompareResponse(samplerSpec, true); + } + + @Test + public void testWithInputRowParser() throws IOException + { + insertData(generateRecords(TOPIC)); + + ObjectMapper objectMapper = new DefaultObjectMapper(); + TimestampSpec timestampSpec = new TimestampSpec("timestamp", "iso", null); + DimensionsSpec dimensionsSpec = new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ) + ); + InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8"); + + DataSchema dataSchema = new DataSchema( + "test_ds", + objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class), + new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + }, + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), + null, + objectMapper + ); + + KafkaSupervisorSpec supervisorSpec = new KafkaSupervisorSpec( + null, + dataSchema, + null, + new KafkaSupervisorIOConfig( + TOPIC, + null, + null, + null, + null, + kafkaServer.consumerProperties(), + null, + null, + null, + null, + true, + null, + null, + null, + null, + null, + null + ), + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + + KafkaSamplerSpec samplerSpec = new KafkaSamplerSpec( + supervisorSpec, + new SamplerConfig(5, null, null, null), + new InputSourceSampler(new DefaultObjectMapper()), + OBJECT_MAPPER + ); + + runSamplerAndCompareResponse(samplerSpec, false); + } + + private static void runSamplerAndCompareResponse(SamplerSpec samplerSpec, boolean useInputFormat) + { SamplerResponse response = samplerSpec.sample(); Assert.assertEquals(5, response.getNumRowsRead()); @@ -258,7 +346,7 @@ public class KafkaSamplerSpecTest extends InitializedNullHandlingTest null, null, true, - "Unable to parse row [unparseable] into JSON" + "Unable to parse row [unparseable]" + (useInputFormat ? " into JSON" : "") ), it.next()); Assert.assertFalse(it.hasNext()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java index 9e7d76edddb..a166863b0de 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java @@ -24,14 +24,18 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.druid.client.indexing.SamplerResponse; +import org.apache.druid.client.indexing.SamplerSpec; import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FloatDimensionSchema; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorIOConfig; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec; @@ -54,10 +58,12 @@ import org.easymock.EasyMockSupport; import org.junit.Assert; import org.junit.Test; +import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.Map; public class KinesisSamplerSpecTest extends EasyMockSupport { @@ -112,23 +118,8 @@ public class KinesisSamplerSpecTest extends EasyMockSupport } @Test(timeout = 10_000L) - public void testSample() throws Exception + public void testSample() throws InterruptedException { - EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).once(); - - recordSupplier.assign(ImmutableSet.of(StreamPartition.of(STREAM, SHARD_ID))); - EasyMock.expectLastCall().once(); - - recordSupplier.seekToEarliest(ImmutableSet.of(StreamPartition.of(STREAM, SHARD_ID))); - EasyMock.expectLastCall().once(); - - EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(STREAM)).once(); - - recordSupplier.close(); - EasyMock.expectLastCall().once(); - - replayAll(); - KinesisSupervisorSpec supervisorSpec = new KinesisSupervisorSpec( null, DATA_SCHEMA, @@ -176,6 +167,104 @@ public class KinesisSamplerSpecTest extends EasyMockSupport null ); + runSamplerAndCompareResponse(samplerSpec, true); + } + + @Test + public void testSampleWithInputRowParser() throws IOException, InterruptedException + { + ObjectMapper objectMapper = new DefaultObjectMapper(); + TimestampSpec timestampSpec = new TimestampSpec("timestamp", "iso", null); + DimensionsSpec dimensionsSpec = new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ) + ); + InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8"); + + DataSchema dataSchema = new DataSchema( + "test_ds", + objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class), + new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + }, + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), + null, + objectMapper + ); + + KinesisSupervisorSpec supervisorSpec = new KinesisSupervisorSpec( + null, + dataSchema, + null, + new KinesisSupervisorIOConfig( + STREAM, + null, + null, + null, + null, + null, + null, + null, + null, + true, + null, + null, + null, + null, + null, + null, + null, + null, + null, + false + ), + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + + KinesisSamplerSpec samplerSpec = new TestableKinesisSamplerSpec( + supervisorSpec, + new SamplerConfig(5, null, null, null), + new InputSourceSampler(new DefaultObjectMapper()), + null + ); + + runSamplerAndCompareResponse(samplerSpec, false); + } + + private void runSamplerAndCompareResponse(SamplerSpec samplerSpec, boolean useInputFormat) throws InterruptedException + { + EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).once(); + + recordSupplier.assign(ImmutableSet.of(StreamPartition.of(STREAM, SHARD_ID))); + EasyMock.expectLastCall().once(); + + recordSupplier.seekToEarliest(ImmutableSet.of(StreamPartition.of(STREAM, SHARD_ID))); + EasyMock.expectLastCall().once(); + + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(STREAM)).once(); + + recordSupplier.close(); + EasyMock.expectLastCall().once(); + + replayAll(); + SamplerResponse response = samplerSpec.sample(); verifyAll(); @@ -269,7 +358,7 @@ public class KinesisSamplerSpecTest extends EasyMockSupport null, null, true, - "Unable to parse row [unparseable] into JSON" + "Unable to parse row [unparseable]" + (useInputFormat ? " into JSON" : "") ), it.next()); Assert.assertFalse(it.hasNext()); diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/firehose/s3/S3FirehoseDruidModule.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/firehose/s3/S3FirehoseDruidModule.java deleted file mode 100644 index 6cb5ac5c1d8..00000000000 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/firehose/s3/S3FirehoseDruidModule.java +++ /dev/null @@ -1,48 +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.druid.firehose.s3; - -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.jsontype.NamedType; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.google.common.collect.ImmutableList; -import com.google.inject.Binder; -import org.apache.druid.initialization.DruidModule; - -import java.util.List; - -/** - */ -public class S3FirehoseDruidModule implements DruidModule -{ - @Override - public List getJacksonModules() - { - return ImmutableList.of( - new SimpleModule().registerSubtypes(new NamedType(StaticS3FirehoseFactory.class, "static-s3")) - ); - } - - @Override - public void configure(Binder binder) - { - - } -} diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/firehose/s3/StaticS3FirehoseFactory.java deleted file mode 100644 index 0bf93313dc6..00000000000 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/firehose/s3/StaticS3FirehoseFactory.java +++ /dev/null @@ -1,236 +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.druid.firehose.s3; - -import com.amazonaws.services.s3.model.AmazonS3Exception; -import com.amazonaws.services.s3.model.GetObjectRequest; -import com.amazonaws.services.s3.model.S3Object; -import com.amazonaws.services.s3.model.S3ObjectSummary; -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.storage.s3.S3Utils; -import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; -import org.apache.druid.utils.CompressionUtils; - -import java.io.IOException; -import java.io.InputStream; -import java.net.URI; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; - -/** - * Builds firehoses that read from a predefined list of S3 objects and then dry up. - */ -public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactory -{ - private static final Logger log = new Logger(StaticS3FirehoseFactory.class); - private static final int MAX_LISTING_LENGTH = 1024; - - private final ServerSideEncryptingAmazonS3 s3Client; - private final List uris; - private final List prefixes; - - @JsonCreator - public StaticS3FirehoseFactory( - @JacksonInject ServerSideEncryptingAmazonS3 s3Client, - @JsonProperty("uris") List uris, - @JsonProperty("prefixes") List prefixes, - @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, - @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, - @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, - @JsonProperty("fetchTimeout") Long fetchTimeout, - @JsonProperty("maxFetchRetry") Integer maxFetchRetry - ) - { - super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); - this.s3Client = Preconditions.checkNotNull(s3Client, "s3Client"); - this.uris = uris == null ? new ArrayList<>() : uris; - this.prefixes = prefixes == null ? new ArrayList<>() : prefixes; - - if (!this.uris.isEmpty() && !this.prefixes.isEmpty()) { - throw new IAE("uris and prefixes cannot be used together"); - } - - if (this.uris.isEmpty() && this.prefixes.isEmpty()) { - throw new IAE("uris or prefixes must be specified"); - } - - for (final URI inputURI : this.uris) { - Preconditions.checkArgument("s3".equals(inputURI.getScheme()), "input uri scheme == s3 (%s)", inputURI); - } - - for (final URI inputURI : this.prefixes) { - Preconditions.checkArgument("s3".equals(inputURI.getScheme()), "input uri scheme == s3 (%s)", inputURI); - } - } - - @JsonProperty - public List getUris() - { - return uris; - } - - @JsonProperty("prefixes") - public List getPrefixes() - { - return prefixes; - } - - @Override - protected Collection initObjects() - { - if (!uris.isEmpty()) { - return uris; - } else { - final List objects = new ArrayList<>(); - for (final URI prefix : prefixes) { - final Iterator objectSummaryIterator = S3Utils.objectSummaryIterator( - s3Client, - Collections.singletonList(prefix), - MAX_LISTING_LENGTH - ); - - objectSummaryIterator.forEachRemaining(objects::add); - } - return objects.stream().map(S3Utils::summaryToUri).collect(Collectors.toList()); - } - } - - @Override - protected InputStream openObjectStream(URI object) throws IOException - { - try { - // Get data of the given object and open an input stream - final String bucket = object.getAuthority(); - final String key = S3Utils.extractS3Key(object); - - final S3Object s3Object = s3Client.getObject(bucket, key); - if (s3Object == null) { - throw new ISE("Failed to get an s3 object for bucket[%s] and key[%s]", bucket, key); - } - return s3Object.getObjectContent(); - } - catch (AmazonS3Exception e) { - throw new IOException(e); - } - } - - @Override - protected InputStream openObjectStream(URI object, long start) throws IOException - { - final String bucket = object.getAuthority(); - final String key = S3Utils.extractS3Key(object); - - final GetObjectRequest request = new GetObjectRequest(bucket, key); - request.setRange(start); - try { - final S3Object s3Object = s3Client.getObject(request); - if (s3Object == null) { - throw new ISE( - "Failed to get an s3 object for bucket[%s], key[%s], and start[%d]", - bucket, - key, - start - ); - } - return s3Object.getObjectContent(); - } - catch (AmazonS3Exception e) { - throw new IOException(e); - } - } - - @Override - protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException - { - return CompressionUtils.decompress(stream, S3Utils.extractS3Key(object)); - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - StaticS3FirehoseFactory that = (StaticS3FirehoseFactory) o; - - return Objects.equals(uris, that.uris) && - Objects.equals(prefixes, that.prefixes) && - getMaxCacheCapacityBytes() == that.getMaxCacheCapacityBytes() && - getMaxFetchCapacityBytes() == that.getMaxFetchCapacityBytes() && - getPrefetchTriggerBytes() == that.getPrefetchTriggerBytes() && - getFetchTimeout() == that.getFetchTimeout() && - getMaxFetchRetry() == that.getMaxFetchRetry(); - } - - @Override - public int hashCode() - { - return Objects.hash( - uris, - prefixes, - getMaxCacheCapacityBytes(), - getMaxFetchCapacityBytes(), - getPrefetchTriggerBytes(), - getFetchTimeout(), - getMaxFetchRetry() - ); - } - - @Override - protected Predicate getRetryCondition() - { - return S3Utils.S3RETRY; - } - - @Override - public FiniteFirehoseFactory withSplit(InputSplit split) - { - return new StaticS3FirehoseFactory( - s3Client, - Collections.singletonList(split.get()), - null, - getMaxCacheCapacityBytes(), - getMaxFetchCapacityBytes(), - getPrefetchTriggerBytes(), - getFetchTimeout(), - getMaxFetchRetry() - ); - } -} diff --git a/extensions-core/s3-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-core/s3-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule index fcbab8ed256..1a770714861 100644 --- a/extensions-core/s3-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule +++ b/extensions-core/s3-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -15,5 +15,4 @@ org.apache.druid.storage.s3.output.S3StorageConnectorModule org.apache.druid.storage.s3.S3StorageDruidModule -org.apache.druid.firehose.s3.S3FirehoseDruidModule org.apache.druid.data.input.s3.S3InputSourceDruidModule diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/firehose/s3/StaticS3FirehoseFactoryTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/firehose/s3/StaticS3FirehoseFactoryTest.java deleted file mode 100644 index c809bf37040..00000000000 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/firehose/s3/StaticS3FirehoseFactoryTest.java +++ /dev/null @@ -1,115 +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.druid.firehose.s3; - -import com.amazonaws.services.s3.AmazonS3Client; -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.s3.S3InputSourceTest; -import org.apache.druid.storage.s3.NoopServerSideEncryption; -import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; -import org.easymock.EasyMock; -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; -import java.net.URI; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; -import java.util.stream.Collectors; - -/** - */ -public class StaticS3FirehoseFactoryTest -{ - private static final AmazonS3Client S3_CLIENT = EasyMock.createNiceMock(AmazonS3Client.class); - private static final ServerSideEncryptingAmazonS3 SERVICE = new ServerSideEncryptingAmazonS3( - S3_CLIENT, - new NoopServerSideEncryption() - ); - - @Test - public void testSerde() throws Exception - { - final ObjectMapper mapper = S3InputSourceTest.createS3ObjectMapper(); - - final List uris = Arrays.asList( - new URI("s3://foo/bar/file.gz"), - new URI("s3://bar/foo/file2.gz") - ); - - final StaticS3FirehoseFactory factory = new StaticS3FirehoseFactory( - SERVICE, - uris, - null, - 2048L, - 1024L, - 512L, - 100L, - 5 - ); - - final StaticS3FirehoseFactory outputFact = mapper.readValue( - mapper.writeValueAsString(factory), - StaticS3FirehoseFactory.class - ); - - Assert.assertEquals(factory, outputFact); - } - - @Test - public void testWithSplit() throws IOException - { - final List uris = Arrays.asList( - URI.create("s3://foo/bar/file.gz"), - URI.create("s3://bar/foo/file2.gz") - ); - uris.sort(Comparator.comparing(URI::toString)); - - final StaticS3FirehoseFactory factory = new StaticS3FirehoseFactory( - SERVICE, - uris, - null, - 2048L, - 1024L, - 512L, - 100L, - 5 - ); - final List> subFactories = factory - .getSplits(null) - .map(factory::withSplit) - .sorted(Comparator.comparing(eachFactory -> { - final StaticS3FirehoseFactory staticS3FirehoseFactory = (StaticS3FirehoseFactory) eachFactory; - return staticS3FirehoseFactory.getUris().toString(); - })) - .collect(Collectors.toList()); - - Assert.assertEquals(uris.size(), subFactories.size()); - for (int i = 0; i < uris.size(); i++) { - final StaticS3FirehoseFactory staticS3FirehoseFactory = (StaticS3FirehoseFactory) subFactories.get(i); - final List subFactoryUris = staticS3FirehoseFactory.getUris(); - Assert.assertEquals(1, subFactoryUris.size()); - Assert.assertEquals(uris.get(i), subFactoryUris.get(0)); - } - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceFirehoseModule.java b/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceFirehoseModule.java index 83d9825d14f..8e495221099 100644 --- a/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceFirehoseModule.java +++ b/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceFirehoseModule.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; -import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory; import org.apache.druid.initialization.DruidModule; import org.apache.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; @@ -38,8 +37,7 @@ public class IndexingServiceFirehoseModule implements DruidModule return ImmutableList.of( new SimpleModule("IndexingServiceFirehoseModule") .registerSubtypes( - new NamedType(EventReceiverFirehoseFactory.class, "receiver"), - new NamedType(IngestSegmentFirehoseFactory.class, "ingestSegment") + new NamedType(EventReceiverFirehoseFactory.class, "receiver") ) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/ReingestionTimelineUtils.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/ReingestionTimelineUtils.java deleted file mode 100644 index 8714fa6933a..00000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/ReingestionTimelineUtils.java +++ /dev/null @@ -1,140 +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.druid.indexing.common; - -import com.google.common.collect.BiMap; -import com.google.common.collect.HashBiMap; -import com.google.common.collect.Lists; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.TimelineObjectHolder; -import org.apache.druid.timeline.partition.PartitionChunk; - -import javax.annotation.Nullable; -import javax.validation.constraints.NotNull; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -/** - * @deprecated only used by {@link org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory} - */ -@Deprecated -public class ReingestionTimelineUtils -{ - /** - * @param timelineSegments A list of timeline objects, such as that returned by VersionedIntervalTimeline.lookup(). - * @param excludeDimensions Dimensions to be excluded - * @return A list of all the unique dimension column names present in the segments within timelineSegments - */ - public static List getUniqueDimensions( - List> timelineSegments, - @Nullable Set excludeDimensions - ) - { - final BiMap uniqueDims = HashBiMap.create(); - - // Here, we try to retain the order of dimensions as they were specified since the order of dimensions may be - // optimized for performance. - // Dimensions are extracted from the recent segments to olders because recent segments are likely to be queried more - // frequently, and thus the performance should be optimized for recent ones rather than old ones. - - // timelineSegments are sorted in order of interval - int index = 0; - for (TimelineObjectHolder timelineHolder : Lists.reverse(timelineSegments)) { - for (PartitionChunk chunk : timelineHolder.getObject()) { - for (String dimension : chunk.getObject().getDimensions()) { - if (!uniqueDims.containsKey(dimension) && - (excludeDimensions == null || !excludeDimensions.contains(dimension))) { - uniqueDims.put(dimension, index++); - } - } - } - } - - final BiMap orderedDims = uniqueDims.inverse(); - return IntStream.range(0, orderedDims.size()) - .mapToObj(orderedDims::get) - .collect(Collectors.toList()); - } - - /** - * @param timelineSegments A list of timeline objects, such as that returned by VersionedIntervalTimeline.lookup(). - * @return A list of all the unique metric column names present in the segments within timelineSegments - */ - public static List getUniqueMetrics(List> timelineSegments) - { - final BiMap uniqueMetrics = HashBiMap.create(); - - // Here, we try to retain the order of metrics as they were specified. Metrics are extracted from the recent - // segments to olders. - - // timelineSegments are sorted in order of interval - int[] index = {0}; - for (TimelineObjectHolder timelineHolder : Lists.reverse(timelineSegments)) { - for (PartitionChunk chunk : timelineHolder.getObject()) { - for (String metric : chunk.getObject().getMetrics()) { - uniqueMetrics.computeIfAbsent( - metric, - k -> { - return index[0]++; - } - ); - } - } - } - - final BiMap orderedMetrics = uniqueMetrics.inverse(); - return IntStream.range(0, orderedMetrics.size()) - .mapToObj(orderedMetrics::get) - .collect(Collectors.toList()); - } - - /** - * Utility function to get dimensions that should be ingested. The preferred order is - * - Explicit dimensions if they are provided. - * - Custom dimensions are provided in the inputSpec. - * - Calculate dimensions from the timeline but exclude any dimension exclusions. - * - * @param explicitDimensions sent as part of the re-ingestion InputSource. - * @param dimensionsSpec from the provided ingestion spec. - * @param timeLineSegments for the datasource that is being read. - * @return - */ - public static List getDimensionsToReingest( - @Nullable List explicitDimensions, - @NotNull DimensionsSpec dimensionsSpec, - @NotNull List> timeLineSegments) - { - final List dims; - if (explicitDimensions != null) { - dims = explicitDimensions; - } else if (dimensionsSpec.hasCustomDimensions()) { - dims = dimensionsSpec.getDimensionNames(); - } else { - dims = ReingestionTimelineUtils.getUniqueDimensions( - timeLineSegments, - dimensionsSpec.getDimensionExclusions() - ); - } - return dims; - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index 2f19c1a5a0a..a4cd183c01b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -25,7 +25,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.errorprone.annotations.concurrent.GuardedBy; import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; -import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputSource; @@ -44,8 +43,6 @@ import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.indexing.common.task.batch.MaxAllowedLocksExceededException; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; -import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory; -import org.apache.druid.indexing.firehose.WindowedSegmentId; import org.apache.druid.indexing.input.InputRowSchemas; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.IAE; @@ -85,7 +82,6 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -570,13 +566,9 @@ public abstract class AbstractBatchIndexTask extends AbstractTask } /** - * If the given firehoseFactory is {@link IngestSegmentFirehoseFactory}, then it finds the segments to lock - * from the firehoseFactory. This is because those segments will be read by this task no matter what segments would be - * filtered by intervalsToRead, so they need to be locked. *

- * However, firehoseFactory is not IngestSegmentFirehoseFactory, it means this task will overwrite some segments - * with data read from some input source outside of Druid. As a result, only the segments falling in intervalsToRead - * should be locked. + * This task will overwrite some segments with data read from input source outside of Druid. + * As a result, only the segments falling in intervalsToRead should be locked. *

* The order of segments within the returned list is unspecified, but each segment is guaranteed to appear in the list * only once. @@ -584,48 +576,14 @@ public abstract class AbstractBatchIndexTask extends AbstractTask protected static List findInputSegments( String dataSource, TaskActionClient actionClient, - List intervalsToRead, - FirehoseFactory firehoseFactory + List intervalsToRead ) throws IOException { - if (firehoseFactory instanceof IngestSegmentFirehoseFactory) { - // intervalsToRead is ignored here. - final List inputSegments = ((IngestSegmentFirehoseFactory) firehoseFactory).getSegments(); - if (inputSegments == null) { - final Interval inputInterval = Preconditions.checkNotNull( - ((IngestSegmentFirehoseFactory) firehoseFactory).getInterval(), - "input interval" - ); - - return ImmutableList.copyOf( - actionClient.submit( - new RetrieveUsedSegmentsAction(dataSource, inputInterval, null, Segments.ONLY_VISIBLE) - ) - ); - } else { - final List inputSegmentIds = - inputSegments.stream().map(WindowedSegmentId::getSegmentId).collect(Collectors.toList()); - final Collection dataSegmentsInIntervals = actionClient.submit( - new RetrieveUsedSegmentsAction( - dataSource, - null, - inputSegments.stream() - .flatMap(windowedSegmentId -> windowedSegmentId.getIntervals().stream()) - .collect(Collectors.toSet()), - Segments.ONLY_VISIBLE - ) - ); - return dataSegmentsInIntervals.stream() - .filter(segment -> inputSegmentIds.contains(segment.getId().toString())) - .collect(Collectors.toList()); - } - } else { - return ImmutableList.copyOf( - actionClient.submit( - new RetrieveUsedSegmentsAction(dataSource, null, intervalsToRead, Segments.ONLY_VISIBLE) - ) - ); - } + return ImmutableList.copyOf( + actionClient.submit( + new RetrieveUsedSegmentsAction(dataSource, null, intervalsToRead, Segments.ONLY_VISIBLE) + ) + ); } /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index ca7fadb24f9..bdfb17ac178 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -112,6 +112,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +@Deprecated public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements ChatHandler { private static final String CTX_KEY_LOOKUP_TIER = "lookupTier"; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index d4b04b82472..1934f572d01 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -32,14 +32,11 @@ import com.google.common.collect.ImmutableList; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.Rows; -import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.hll.HyperLogLogCollector; import org.apache.druid.indexer.Checks; import org.apache.druid.indexer.IngestionState; @@ -269,8 +266,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler return findInputSegments( getDataSource(), taskActionClient, - intervals, - ingestionSchema.ioConfig.firehoseFactory + intervals ); } @@ -486,9 +482,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler .inputIntervals() .isEmpty(); - final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource( - ingestionSchema.getDataSchema().getParser() - ); + final InputSource inputSource = ingestionSchema.getIOConfig().getInputSource(); final File tmpDir = toolbox.getIndexingTmpDir(); @@ -1195,16 +1189,9 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler return inputFormat; } - public InputSource getNonNullInputSource(@Nullable InputRowParser inputRowParser) + public InputSource getNonNullInputSource() { - if (inputSource == null) { - return new FirehoseFactoryToInputSourceAdaptor( - (FiniteFirehoseFactory) firehoseFactory, - inputRowParser - ); - } else { - return inputSource; - } + return Preconditions.checkNotNull(inputSource, "inputSource"); } public InputFormat getNonNullInputFormat() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java index 9c3c5cdb238..a31f7e8edde 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java @@ -78,6 +78,7 @@ import java.util.TimerTask; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ThreadLocalRandom; +@Deprecated public class RealtimeIndexTask extends AbstractTask { public static final String CTX_KEY_LOOKUP_TIER = "lookupTier"; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/InputSourceSplitParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/InputSourceSplitParallelIndexTaskRunner.java index 3f41dba24c1..bfdad011556 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/InputSourceSplitParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/InputSourceSplitParallelIndexTaskRunner.java @@ -20,7 +20,6 @@ package org.apache.druid.indexing.common.task.batch.parallel; import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.SplittableInputSource; @@ -59,9 +58,7 @@ abstract class InputSourceSplitParallelIndexTaskRunner e @Override public final TaskStatus runTask(TaskToolbox toolbox) throws Exception { - final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource( - ingestionSchema.getDataSchema().getParser() - ); + final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource(); final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build( supervisorTaskId, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java index 1d51d6fa9ca..7f781e466ed 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java @@ -22,7 +22,6 @@ package org.apache.druid.indexing.common.task.batch.parallel; import com.google.common.annotations.VisibleForTesting; import org.apache.commons.lang3.mutable.MutableObject; import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.SplittableInputSource; @@ -118,9 +117,7 @@ public class SinglePhaseParallelIndexTaskRunner extends ParallelIndexPhaseRunner context ); this.ingestionSchema = ingestionSchema; - this.baseInputSource = (SplittableInputSource) ingestionSchema.getIOConfig().getNonNullInputSource( - ingestionSchema.getDataSchema().getParser() - ); + this.baseInputSource = (SplittableInputSource) ingestionSchema.getIOConfig().getNonNullInputSource(); } @VisibleForTesting @@ -171,13 +168,9 @@ public class SinglePhaseParallelIndexTaskRunner extends ParallelIndexPhaseRunner { final FirehoseFactory firehoseFactory; final InputSource inputSource; - if (baseInputSource instanceof FirehoseFactoryToInputSourceAdaptor) { - firehoseFactory = ((FirehoseFactoryToInputSourceAdaptor) baseInputSource).getFirehoseFactory().withSplit(split); - inputSource = null; - } else { - firehoseFactory = null; - inputSource = baseInputSource.withSplit(split); - } + firehoseFactory = null; + inputSource = baseInputSource.withSplit(split); + final Map subtaskContext = new HashMap<>(getContext()); return new SinglePhaseSubTaskSpec( getBaseSubtaskSpecName() + "_" + getAndIncrementNextSpecId(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index 1b83562d7c7..03a49efaf59 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -247,9 +247,7 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() ); - final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource( - ingestionSchema.getDataSchema().getParser() - ); + final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource(); final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build( supervisorTaskId, @@ -307,8 +305,7 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand return findInputSegments( getDataSource(), taskActionClient, - intervals, - ingestionSchema.getIOConfig().getFirehoseFactory() + intervals ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java deleted file mode 100644 index f20a0ddd1a1..00000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ /dev/null @@ -1,330 +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.druid.indexing.firehose; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Function; -import com.google.common.base.Preconditions; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import org.apache.druid.client.coordinator.CoordinatorClient; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.SegmentsSplitHintSpec; -import org.apache.druid.data.input.SplitHintSpec; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.indexing.common.ReingestionTimelineUtils; -import org.apache.druid.indexing.common.RetryPolicyFactory; -import org.apache.druid.indexing.common.SegmentCacheManagerFactory; -import org.apache.druid.indexing.input.DruidInputSource; -import org.apache.druid.java.util.common.HumanReadableBytes; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.parsers.ParseException; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.query.filter.DimFilter; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.loading.SegmentCacheManager; -import org.apache.druid.segment.loading.SegmentLoadingException; -import org.apache.druid.segment.realtime.firehose.IngestSegmentFirehose; -import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; -import org.apache.druid.segment.transform.TransformSpec; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.TimelineObjectHolder; -import org.apache.druid.timeline.partition.PartitionChunk; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.stream.Stream; - -/** - * @deprecated use {@link DruidInputSource} instead - */ -@Deprecated -public class IngestSegmentFirehoseFactory implements FiniteFirehoseFactory> -{ - private static final EmittingLogger log = new EmittingLogger(IngestSegmentFirehoseFactory.class); - private final String dataSource; - // Exactly one of interval and segmentIds should be non-null. Typically 'interval' is specified directly - // by the user creating this firehose and 'segmentIds' is used for sub-tasks if it is split for parallel - // batch ingestion. - @Nullable - private final Interval interval; - @Nullable - private final List segmentIds; - private final DimFilter dimFilter; - private final List dimensions; - private final List metrics; - @Nullable - private final Long maxInputSegmentBytesPerTask; - private final IndexIO indexIO; - private final CoordinatorClient coordinatorClient; - private final SegmentCacheManagerFactory segmentCacheManagerFactory; - private final RetryPolicyFactory retryPolicyFactory; - - private List>> splits; - - @JsonCreator - public IngestSegmentFirehoseFactory( - @JsonProperty("dataSource") final String dataSource, - @JsonProperty("interval") @Nullable Interval interval, - // Specifying "segments" is intended only for when this FirehoseFactory has split itself, - // not for direct end user use. - @JsonProperty("segments") @Nullable List segmentIds, - @JsonProperty("filter") DimFilter dimFilter, - @JsonProperty("dimensions") List dimensions, - @JsonProperty("metrics") List metrics, - @JsonProperty("maxInputSegmentBytesPerTask") @Deprecated @Nullable Long maxInputSegmentBytesPerTask, - @JacksonInject IndexIO indexIO, - @JacksonInject CoordinatorClient coordinatorClient, - @JacksonInject SegmentCacheManagerFactory segmentCacheManagerFactory, - @JacksonInject RetryPolicyFactory retryPolicyFactory - ) - { - Preconditions.checkNotNull(dataSource, "dataSource"); - if ((interval == null && segmentIds == null) || (interval != null && segmentIds != null)) { - throw new IAE("Specify exactly one of 'interval' and 'segments'"); - } - this.dataSource = dataSource; - this.interval = interval; - this.segmentIds = segmentIds; - this.dimFilter = dimFilter; - this.dimensions = dimensions; - this.metrics = metrics; - this.maxInputSegmentBytesPerTask = maxInputSegmentBytesPerTask; - this.indexIO = Preconditions.checkNotNull(indexIO, "null IndexIO"); - this.coordinatorClient = Preconditions.checkNotNull(coordinatorClient, "null CoordinatorClient"); - this.segmentCacheManagerFactory = Preconditions.checkNotNull(segmentCacheManagerFactory, "null segmentCacheManagerFactory"); - this.retryPolicyFactory = Preconditions.checkNotNull(retryPolicyFactory, "null RetryPolicyFactory"); - } - - @Override - public FiniteFirehoseFactory> withSplit(InputSplit> split) - { - return new IngestSegmentFirehoseFactory( - dataSource, - null, - split.get(), - dimFilter, - dimensions, - metrics, - maxInputSegmentBytesPerTask, - indexIO, - coordinatorClient, - segmentCacheManagerFactory, - retryPolicyFactory - ); - } - - @JsonProperty - public String getDataSource() - { - return dataSource; - } - - @JsonProperty - @Nullable - public Interval getInterval() - { - return interval; - } - - @JsonProperty - @Nullable - public List getSegments() - { - return segmentIds; - } - - @JsonProperty("filter") - public DimFilter getDimensionsFilter() - { - return dimFilter; - } - - @JsonProperty - public List getDimensions() - { - return dimensions; - } - - @JsonProperty - public List getMetrics() - { - return metrics; - } - - @Nullable - @JsonProperty - public Long getMaxInputSegmentBytesPerTask() - { - return maxInputSegmentBytesPerTask; - } - - @Override - public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) throws ParseException - { - log.debug( - "Connecting firehose: dataSource[%s], interval[%s], segmentIds[%s]", - dataSource, - interval, - segmentIds - ); - - final List> timeLineSegments = getTimeline(); - - // Download all segments locally. - // Note: this requires enough local storage space to fit all of the segments, even though - // IngestSegmentFirehose iterates over the segments in series. We may want to change this - // to download files lazily, perhaps sharing code with PrefetchableTextFilesFirehoseFactory. - final SegmentCacheManager segmentCacheManager = segmentCacheManagerFactory.manufacturate(temporaryDirectory); - Map segmentFileMap = Maps.newLinkedHashMap(); - for (TimelineObjectHolder holder : timeLineSegments) { - for (PartitionChunk chunk : holder.getObject()) { - final DataSegment segment = chunk.getObject(); - - segmentFileMap.computeIfAbsent(segment, k -> { - try { - return segmentCacheManager.getSegmentFiles(segment); - } - catch (SegmentLoadingException e) { - throw new RuntimeException(e); - } - }); - } - } - - final List dims = ReingestionTimelineUtils.getDimensionsToReingest( - dimensions, - inputRowParser.getParseSpec().getDimensionsSpec(), - timeLineSegments - ); - final List metricsList = metrics == null - ? ReingestionTimelineUtils.getUniqueMetrics(timeLineSegments) - : metrics; - - final List adapters = Lists.newArrayList( - Iterables.concat( - Iterables.transform( - timeLineSegments, - new Function, Iterable>() { - @Override - public Iterable apply(final TimelineObjectHolder holder) - { - return - Iterables.transform( - holder.getObject(), - new Function, WindowedStorageAdapter>() { - @Override - public WindowedStorageAdapter apply(final PartitionChunk input) - { - final DataSegment segment = input.getObject(); - try { - return new WindowedStorageAdapter( - new QueryableIndexStorageAdapter( - indexIO.loadIndex( - Preconditions.checkNotNull( - segmentFileMap.get(segment), - "File for segment %s", segment.getId() - ) - ) - ), - holder.getInterval() - ); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - } - ); - } - } - ) - ) - ); - - final TransformSpec transformSpec = TransformSpec.fromInputRowParser(inputRowParser); - return new IngestSegmentFirehose(adapters, transformSpec, dims, metricsList, dimFilter); - } - - private List> getTimeline() - { - if (interval == null) { - return DruidInputSource.getTimelineForSegmentIds(coordinatorClient, dataSource, segmentIds); - } else { - return DruidInputSource.getTimelineForInterval(coordinatorClient, retryPolicyFactory, dataSource, interval); - } - } - - private void initializeSplitsIfNeeded(@Nullable SplitHintSpec splitHintSpec) - { - if (splits != null) { - return; - } - - splits = Lists.newArrayList( - DruidInputSource.createSplits( - coordinatorClient, - retryPolicyFactory, - dataSource, - interval, - splitHintSpec == null - ? new SegmentsSplitHintSpec( - maxInputSegmentBytesPerTask == null - ? null - : new HumanReadableBytes(maxInputSegmentBytesPerTask), - null - ) - : splitHintSpec - ) - ); - } - - @Override - public boolean isSplittable() - { - // Specifying 'segments' to this factory instead of 'interval' is intended primarily for internal use by - // parallel batch injection: we don't need to support splitting a list of segments. - return interval != null; - } - - @Override - public Stream>> getSplits(@Nullable SplitHintSpec splitHintSpec) - { - initializeSplitsIfNeeded(splitHintSpec); - return splits.stream(); - } - - @Override - public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) - { - initializeSplitsIfNeeded(splitHintSpec); - return splits.size(); - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java index 4140e014ef7..a0b84e1e757 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java @@ -25,13 +25,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.client.indexing.SamplerResponse; import org.apache.druid.client.indexing.SamplerSpec; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSource; import org.apache.druid.indexing.common.task.IndexTask; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.segment.indexing.DataSchema; import javax.annotation.Nullable; @@ -61,28 +57,17 @@ public class IndexTaskSamplerSpec implements SamplerSpec Preconditions.checkNotNull(ingestionSpec.getIOConfig(), "[spec.ioConfig] is required"); - if (ingestionSpec.getIOConfig().getInputSource() != null) { - this.inputSource = ingestionSpec.getIOConfig().getInputSource(); - if (ingestionSpec.getIOConfig().getInputSource().needsFormat()) { - this.inputFormat = Preconditions.checkNotNull( - ingestionSpec.getIOConfig().getInputFormat(), - "[spec.ioConfig.inputFormat] is required" - ); - } else { - this.inputFormat = null; - } + this.inputSource = Preconditions.checkNotNull( + ingestionSpec.getIOConfig().getInputSource(), + "[spec.ioConfig.inputSource] is required" + ); + + if (inputSource.needsFormat()) { + this.inputFormat = Preconditions.checkNotNull( + ingestionSpec.getIOConfig().getInputFormat(), + "[spec.ioConfig.inputFormat] is required" + ); } else { - final FirehoseFactory firehoseFactory = Preconditions.checkNotNull( - ingestionSpec.getIOConfig().getFirehoseFactory(), - "[spec.ioConfig.firehose] is required" - ); - if (!(firehoseFactory instanceof FiniteFirehoseFactory)) { - throw new IAE("firehose should be an instance of FiniteFirehoseFactory"); - } - this.inputSource = new FirehoseFactoryToInputSourceAdaptor( - (FiniteFirehoseFactory) firehoseFactory, - ingestionSpec.getDataSchema().getParser() - ); this.inputFormat = null; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java index f27cdb50fcc..af850dd1050 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java @@ -23,19 +23,20 @@ import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import org.apache.druid.client.indexing.SamplerResponse; import org.apache.druid.client.indexing.SamplerSpec; -import org.apache.druid.data.input.ByteBufferInputRowParser; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; +import org.apache.druid.data.input.AbstractInputSource; import org.apache.druid.data.input.InputEntity; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; +import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.InputStats; import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; import org.apache.druid.indexing.overlord.sampler.SamplerConfig; @@ -88,10 +89,7 @@ public abstract class SeekableStreamSamplerSpec recordSupplier; @@ -120,18 +118,18 @@ public abstract class SeekableStreamSamplerSpec createRecordSupplier(); - private class SeekableStreamSamplerFirehoseFactory implements FiniteFirehoseFactory + private class SeekableStreamSamplerInputSource extends AbstractInputSource implements SplittableInputSource { - @Override - public Firehose connect(ByteBufferInputRowParser parser, @Nullable File temporaryDirectory) + private final InputRowParser parser; + + public SeekableStreamSamplerInputSource(InputRowParser parser) { - throw new UnsupportedOperationException(); + this.parser = parser; } - @Override - public Firehose connectForSampler(ByteBufferInputRowParser parser, @Nullable File temporaryDirectory) + public InputRowParser getParser() { - return new SeekableStreamSamplerFirehose(parser); + return parser; } @Override @@ -141,30 +139,42 @@ public abstract class SeekableStreamSamplerSpec> getSplits(@Nullable SplitHintSpec splitHintSpec) + public Stream createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) { throw new UnsupportedOperationException(); } @Override - public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) + public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) { throw new UnsupportedOperationException(); } @Override - public FiniteFirehoseFactory withSplit(InputSplit split) + public SplittableInputSource withSplit(InputSplit split) { throw new UnsupportedOperationException(); } + + @Override + public boolean needsFormat() + { + return false; + } + + @Override + protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) + { + return new SeekableStreamSamplerInputSourceReader(parser); + } } - private class SeekableStreamSamplerFirehose implements Firehose + private class SeekableStreamSamplerInputSourceReader implements InputSourceReader { private final InputRowParser parser; private final CloseableIterator entityIterator; - protected SeekableStreamSamplerFirehose(InputRowParser parser) + public SeekableStreamSamplerInputSourceReader(InputRowParser parser) { this.parser = parser; if (parser instanceof StringInputRowParser) { @@ -181,47 +191,81 @@ public abstract class SeekableStreamSamplerSpec read() { - return entityIterator.hasNext(); - } + return new CloseableIterator() + { - @Override - public InputRow nextRow() - { - throw new UnsupportedOperationException(); - } - - @Override - public InputRowListPlusRawValues nextRowWithRaw() - { - final ByteBuffer bb = ((ByteEntity) entityIterator.next()).getBuffer(); - - final Map rawColumns; - try { - if (parser instanceof StringInputRowParser) { - rawColumns = ((StringInputRowParser) parser).buildStringKeyMap(bb); - } else { - rawColumns = null; + @Override + public boolean hasNext() + { + return entityIterator.hasNext(); } - } - catch (ParseException e) { - return InputRowListPlusRawValues.of(null, e); - } - try { - final List rows = parser.parseBatch(bb); - return InputRowListPlusRawValues.of(rows.isEmpty() ? null : rows, rawColumns); - } - catch (ParseException e) { - return InputRowListPlusRawValues.of(rawColumns, e); - } + @Override + public InputRow next() + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws IOException + { + entityIterator.close(); + } + }; } @Override - public void close() throws IOException + public CloseableIterator read(InputStats inputStats) { - entityIterator.close(); + return null; + } + + @Override + public CloseableIterator sample() + { + return new CloseableIterator() + { + @Override + public boolean hasNext() + { + return entityIterator.hasNext(); + } + + @Override + public InputRowListPlusRawValues next() + { + final ByteBuffer bb = ((ByteEntity) entityIterator.next()).getBuffer(); + + final Map rawColumns; + try { + if (parser instanceof StringInputRowParser) { + rawColumns = ((StringInputRowParser) parser).buildStringKeyMap(bb); + } else { + rawColumns = null; + } + } + catch (ParseException e) { + return InputRowListPlusRawValues.of(null, e); + } + + try { + bb.position(0); + final List rows = parser.parseBatch(bb); + return InputRowListPlusRawValues.of(rows.isEmpty() ? null : rows, rawColumns); + } + catch (ParseException e) { + return InputRowListPlusRawValues.of(rawColumns, e); + } + } + + @Override + public void close() throws IOException + { + entityIterator.close(); + } + }; } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestRealtimeTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java similarity index 61% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/TestRealtimeTask.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java index eca5ed3557a..0dbc6f718c8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestRealtimeTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java @@ -24,26 +24,27 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.common.task.RealtimeIndexTask; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.apache.druid.segment.realtime.FireDepartment; -import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; import java.io.File; /** */ -@JsonTypeName("test_realtime") -public class TestRealtimeTask extends RealtimeIndexTask +@JsonTypeName("test_index") +public class TestIndexTask extends IndexTask { private final TaskStatus status; @JsonCreator - public TestRealtimeTask( + public TestIndexTask( @JsonProperty("id") String id, @JsonProperty("resource") TaskResource taskResource, @JsonProperty("dataSource") String dataSource, @@ -54,13 +55,42 @@ public class TestRealtimeTask extends RealtimeIndexTask super( id, taskResource, - new FireDepartment( + new IndexIngestionSpec( new DataSchema(dataSource, null, new AggregatorFactory[]{}, null, null, mapper), - new RealtimeIOConfig( - new LocalFirehoseFactory(new File("lol"), "rofl", null), - (schema, config, metrics) -> null + new IndexTask.IndexIOConfig( + null, + new LocalInputSource(new File("lol"), "rofl"), + new JsonInputFormat(null, null, null, null, null), + false, + false ), - null + + new IndexTask.IndexTuningConfig( + null, + null, + null, + 10, + null, + null, + null, + null, + null, + null, + new DynamicPartitionsSpec(10000, null), + new IndexSpec(), + null, + 3, + false, + null, + null, + null, + null, + null, + null, + null, + null, + null + ) ), null ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 3713f532574..702074aad93 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -32,7 +32,6 @@ import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.ParseSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; @@ -48,7 +47,6 @@ import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.CompactionTask.Builder; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; -import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; @@ -71,7 +69,6 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.LocalDataSegmentPuller; @@ -1489,96 +1486,6 @@ public class CompactionTaskRunTest extends IngestionTestBase Assert.assertEquals(TaskState.FAILED, compactionResult.lhs.getStatusCode()); } - /** - * Run a regular index task that's equivalent to the compaction task in {@link #testRunWithDynamicPartitioning()}, - * using {@link IngestSegmentFirehoseFactory}. - *

- * This is not entirely CompactionTask related, but it's similar conceptually and it requires - * similar setup to what this test suite already has. - *

- * It could be moved to a separate test class if needed. - */ - @Test - public void testRunRegularIndexTaskWithIngestSegmentFirehose() throws Exception - { - runIndexTask(); - - IndexTask indexTask = new IndexTask( - null, - null, - new IndexTask.IndexIngestionSpec( - new DataSchema( - "test", - getObjectMapper().convertValue( - new StringInputRowParser( - DEFAULT_PARSE_SPEC, - null - ), - Map.class - ), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.MINUTE, - null - ), - null, - getObjectMapper() - ), - new IndexTask.IndexIOConfig( - new IngestSegmentFirehoseFactory( - DATA_SOURCE, - Intervals.of("2014-01-01/2014-01-02"), - null, - null, - null, - null, - null, - getIndexIO(), - coordinatorClient, - segmentCacheManagerFactory, - RETRY_POLICY_FACTORY - ), - false, - false - ), - IndexTaskTest.createTuningConfig(5000000, null, null, Long.MAX_VALUE, null, false, true) - ), - null - ); - - // This is a regular index so we need to explicitly add this context to store the CompactionState - indexTask.addToContext(Tasks.STORE_COMPACTION_STATE_KEY, true); - - final Pair> resultPair = runTask(indexTask); - - Assert.assertTrue(resultPair.lhs.isSuccess()); - - final List segments = resultPair.rhs; - Assert.assertEquals(3, segments.size()); - - for (int i = 0; i < 3; i++) { - Assert.assertEquals( - Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), - segments.get(i).getInterval() - ); - Assert.assertEquals( - getDefaultCompactionState(Granularities.HOUR, Granularities.MINUTE, ImmutableList.of()), - segments.get(i).getLastCompactionState() - ); - if (lockGranularity == LockGranularity.SEGMENT) { - Assert.assertEquals( - new NumberedOverwriteShardSpec(32768, 0, 2, (short) 1, (short) 1), - segments.get(i).getShardSpec() - ); - } else { - Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec()); - } - } - } - private Pair> runIndexTask() throws Exception { return runIndexTask(null, null, false); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java index 6af2e194c1f..7555c6b1e37 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java @@ -20,11 +20,8 @@ package org.apache.druid.indexing.common.task; import com.google.common.collect.ImmutableMap; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.NoopFirehoseFactory; import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.NoopInputSource; -import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig; import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import org.apache.druid.java.util.common.granularity.Granularities; @@ -91,56 +88,4 @@ public class IndexIngestionSpecTest null ); } - - @Test - public void testFirehoseAndInputSource() - { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage( - "At most one of [Property{name='firehose', value=NoopFirehoseFactory{}}, Property{name='inputSource'" - ); - final IndexIngestionSpec spec = new IndexIngestionSpec( - new DataSchema( - "dataSource", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[0], - new ArbitraryGranularitySpec(Granularities.NONE, null), - null - ), - new IndexIOConfig( - new NoopFirehoseFactory(), - new NoopInputSource(), - null, - null, - null - ), - null - ); - } - - @Test - public void testFirehoseAndInputFormat() - { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("Cannot use firehose and inputFormat together."); - final IndexIngestionSpec spec = new IndexIngestionSpec( - new DataSchema( - "dataSource", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[0], - new ArbitraryGranularitySpec(Granularities.NONE, null), - null - ), - new IndexIOConfig( - new NoopFirehoseFactory(), - null, - new NoopInputFormat(), - null, - null - ), - null - ); - } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 469eaff4dab..5b6dcbfcc42 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -38,7 +38,6 @@ import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; @@ -87,7 +86,6 @@ import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLocalCacheManager; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; import org.apache.druid.segment.realtime.plumber.NoopSegmentHandoffNotifierFactory; import org.apache.druid.segment.transform.ExpressionTransform; @@ -956,7 +954,9 @@ public class IndexTaskTest extends IngestionTestBase ingestionSpec = createIngestionSpec( jsonMapper, tmpDir, - new CSVParseSpec(timestampSpec, DimensionsSpec.EMPTY, null, columns, true, 0), + timestampSpec, + DimensionsSpec.EMPTY, + new CsvInputFormat(columns, null, null, true, 0), null, null, tuningConfig, @@ -967,9 +967,7 @@ public class IndexTaskTest extends IngestionTestBase ingestionSpec = createIngestionSpec( jsonMapper, tmpDir, - timestampSpec, - DimensionsSpec.EMPTY, - new CsvInputFormat(columns, null, null, true, 0), + new CSVParseSpec(timestampSpec, DimensionsSpec.EMPTY, null, columns, true, 0), null, null, tuningConfig, @@ -1484,12 +1482,6 @@ public class IndexTaskTest extends IngestionTestBase false, false ); - expectedMessages = ImmutableList.of( - StringUtils.format( - "Timestamp[unparseable] is unparseable! Event: {time=unparseable, d=a, val=1} (Path: %s, Record: 1, Line: 2)", - tmpFile.toURI() - ) - ); } else { indexIngestionSpec = createIngestionSpec( jsonMapper, @@ -1501,11 +1493,14 @@ public class IndexTaskTest extends IngestionTestBase false, false ); - expectedMessages = ImmutableList.of( - "Timestamp[unparseable] is unparseable! Event: {time=unparseable, d=a, val=1}" - ); } + expectedMessages = ImmutableList.of( + StringUtils.format( + "Timestamp[unparseable] is unparseable! Event: {time=unparseable, d=a, val=1} (Path: %s, Record: 1, Line: 2)", + tmpFile.toURI() + ) + ); IndexTask indexTask = new IndexTask( null, null, @@ -1625,13 +1620,12 @@ public class IndexTaskTest extends IngestionTestBase IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); - final int processedBytes = useInputFormatApi ? 657 : 0; Map expectedMetrics = ImmutableMap.of( RowIngestionMeters.DETERMINE_PARTITIONS, ImmutableMap.of( RowIngestionMeters.PROCESSED_WITH_ERROR, 0, RowIngestionMeters.PROCESSED, 4, - RowIngestionMeters.PROCESSED_BYTES, processedBytes, + RowIngestionMeters.PROCESSED_BYTES, 657, RowIngestionMeters.UNPARSEABLE, 4, RowIngestionMeters.THROWN_AWAY, 1 ), @@ -1639,7 +1633,7 @@ public class IndexTaskTest extends IngestionTestBase ImmutableMap.of( RowIngestionMeters.PROCESSED_WITH_ERROR, 3, RowIngestionMeters.PROCESSED, 1, - RowIngestionMeters.PROCESSED_BYTES, processedBytes, + RowIngestionMeters.PROCESSED_BYTES, 657, RowIngestionMeters.UNPARSEABLE, 4, RowIngestionMeters.THROWN_AWAY, 1 ) @@ -1650,36 +1644,24 @@ public class IndexTaskTest extends IngestionTestBase ParseExceptionReport.forPhase(reportData, RowIngestionMeters.BUILD_SEGMENTS); List expectedMessages; - if (useInputFormatApi) { - expectedMessages = Arrays.asList( - StringUtils.format("Unable to parse row [this is not JSON] (Path: %s, Record: 6, Line: 9)", tmpFile.toURI()), - StringUtils.format( - "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 6, Line: 8)", - tmpFile.toURI() - ), - StringUtils.format( - "Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}] (Path: %s, Record: 5, Line: 6)", - tmpFile.toURI() - ), - "Unable to parse value[notnumber] for field[val]", - "could not convert value [notnumber] to float", - "could not convert value [notnumber] to long", - StringUtils.format( - "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 1)", - tmpFile.toURI() - ) - ); - } else { - expectedMessages = Arrays.asList( - "Unable to parse row [this is not JSON]", - "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}", - "Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}]", - "Unable to parse value[notnumber] for field[val]", - "could not convert value [notnumber] to float", - "could not convert value [notnumber] to long", - "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}" - ); - } + expectedMessages = Arrays.asList( + StringUtils.format("Unable to parse row [this is not JSON] (Path: %s, Record: 6, Line: 9)", tmpFile.toURI()), + StringUtils.format( + "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 6, Line: 8)", + tmpFile.toURI() + ), + StringUtils.format( + "Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}] (Path: %s, Record: 5, Line: 6)", + tmpFile.toURI() + ), + "Unable to parse value[notnumber] for field[val]", + "could not convert value [notnumber] to float", + "could not convert value [notnumber] to long", + StringUtils.format( + "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 1)", + tmpFile.toURI() + ) + ); Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); @@ -1697,30 +1679,21 @@ public class IndexTaskTest extends IngestionTestBase parseExceptionReport = ParseExceptionReport.forPhase(reportData, RowIngestionMeters.DETERMINE_PARTITIONS); - if (useInputFormatApi) { - expectedMessages = Arrays.asList( - StringUtils.format("Unable to parse row [this is not JSON] (Path: %s, Record: 6, Line: 9)", tmpFile.toURI()), - StringUtils.format( - "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 6, Line: 8)", - tmpFile.toURI() - ), - StringUtils.format( - "Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}] (Path: %s, Record: 5, Line: 6)", - tmpFile.toURI() - ), - StringUtils.format( - "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 1)", - tmpFile.toURI() - ) - ); - } else { - expectedMessages = Arrays.asList( - "Unable to parse row [this is not JSON]", - "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}", - "Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}]", - "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}" - ); - } + expectedMessages = Arrays.asList( + StringUtils.format("Unable to parse row [this is not JSON] (Path: %s, Record: 6, Line: 9)", tmpFile.toURI()), + StringUtils.format( + "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 6, Line: 8)", + tmpFile.toURI() + ), + StringUtils.format( + "Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}] (Path: %s, Record: 5, Line: 6)", + tmpFile.toURI() + ), + StringUtils.format( + "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 1)", + tmpFile.toURI() + ) + ); Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages()); @@ -1802,20 +1775,6 @@ public class IndexTaskTest extends IngestionTestBase false, false ); - expectedMessages = Arrays.asList( - StringUtils.format( - "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 3, Line: 6)", - tmpFile.toURI() - ), - StringUtils.format( - "Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 2, Line: 4)", - tmpFile.toURI() - ), - StringUtils.format( - "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 2)", - tmpFile.toURI() - ) - ); } else { ingestionSpec = createIngestionSpec( jsonMapper, @@ -1827,13 +1786,22 @@ public class IndexTaskTest extends IngestionTestBase false, false ); - expectedMessages = Arrays.asList( - "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}", - "Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1}", - "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}" - ); } + expectedMessages = Arrays.asList( + StringUtils.format( + "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 3, Line: 6)", + tmpFile.toURI() + ), + StringUtils.format( + "Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 2, Line: 4)", + tmpFile.toURI() + ), + StringUtils.format( + "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 2)", + tmpFile.toURI() + ) + ); IndexTask indexTask = new IndexTask( null, null, @@ -1860,9 +1828,9 @@ public class IndexTaskTest extends IngestionTestBase ImmutableMap.of( RowIngestionMeters.PROCESSED_WITH_ERROR, 0, RowIngestionMeters.PROCESSED, 1, - RowIngestionMeters.PROCESSED_BYTES, useInputFormatApi ? 182 : 0, + RowIngestionMeters.PROCESSED_BYTES, 182, RowIngestionMeters.UNPARSEABLE, 3, - RowIngestionMeters.THROWN_AWAY, useInputFormatApi ? 1 : 2 + RowIngestionMeters.THROWN_AWAY, 1 ) ); @@ -1949,11 +1917,6 @@ public class IndexTaskTest extends IngestionTestBase false, false ); - expectedMessages = Arrays.asList( - StringUtils.format("Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 3, Line: 6)", tmpFile.toURI()), - StringUtils.format("Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 2, Line: 4)", tmpFile.toURI()), - StringUtils.format("Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 2)", tmpFile.toURI()) - ); } else { ingestionSpec = createIngestionSpec( jsonMapper, @@ -1965,13 +1928,13 @@ public class IndexTaskTest extends IngestionTestBase false, false ); - expectedMessages = Arrays.asList( - "Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}", - "Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1}", - "Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}" - ); } + expectedMessages = Arrays.asList( + StringUtils.format("Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 3, Line: 6)", tmpFile.toURI()), + StringUtils.format("Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 2, Line: 4)", tmpFile.toURI()), + StringUtils.format("Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 2)", tmpFile.toURI()) + ); IndexTask indexTask = new IndexTask( null, null, @@ -1990,9 +1953,9 @@ public class IndexTaskTest extends IngestionTestBase ImmutableMap.of( RowIngestionMeters.PROCESSED_WITH_ERROR, 0, RowIngestionMeters.PROCESSED, 1, - RowIngestionMeters.PROCESSED_BYTES, useInputFormatApi ? 182 : 0, + RowIngestionMeters.PROCESSED_BYTES, 182, RowIngestionMeters.UNPARSEABLE, 3, - RowIngestionMeters.THROWN_AWAY, useInputFormatApi ? 1 : 2 + RowIngestionMeters.THROWN_AWAY, 1 ), RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( @@ -2135,12 +2098,6 @@ public class IndexTaskTest extends IngestionTestBase false, false ); - expectedMessages = ImmutableList.of( - StringUtils.format( - "Timestamp[null] is unparseable! Event: {column_1=2014-01-01T00:00:10Z, column_2=a, column_3=1} (Path: %s, Record: 1, Line: 2)", - tmpFile.toURI() - ) - ); } else { ingestionSpec = createIngestionSpec( jsonMapper, @@ -2152,11 +2109,14 @@ public class IndexTaskTest extends IngestionTestBase false, false ); - expectedMessages = ImmutableList.of( - "Timestamp[null] is unparseable! Event: {column_1=2014-01-01T00:00:10Z, column_2=a, column_3=1}" - ); } + expectedMessages = ImmutableList.of( + StringUtils.format( + "Timestamp[null] is unparseable! Event: {column_1=2014-01-01T00:00:10Z, column_2=a, column_3=1} (Path: %s, Record: 1, Line: 2)", + tmpFile.toURI() + ) + ); IndexTask indexTask = new IndexTask( null, null, @@ -2916,16 +2876,12 @@ public class IndexTaskTest extends IngestionTestBase tuningConfig ); } else { + parseSpec = parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC; return new IndexIngestionSpec( new DataSchema( DATASOURCE, - objectMapper.convertValue( - new StringInputRowParser( - parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC, - null - ), - Map.class - ), + parseSpec.getTimestampSpec(), + parseSpec.getDimensionsSpec(), new AggregatorFactory[]{ new LongSumAggregatorFactory("val", "val") }, @@ -2935,14 +2891,13 @@ public class IndexTaskTest extends IngestionTestBase Collections.singletonList(Intervals.of("2014/2015")) ), transformSpec, + null, objectMapper ), new IndexIOConfig( - new LocalFirehoseFactory( - baseDir, - "druid*", - null - ), + null, + new LocalInputSource(baseDir, "druid*"), + createInputFormatFromParseSpec(parseSpec), appendToExisting, dropExisting ), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 99dc6a9f179..729e74bbafe 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -23,6 +23,18 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.impl.CSVParseSpec; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DelimitedInputFormat; +import org.apache.druid.data.input.impl.DelimitedParseSpec; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.ParseSpec; +import org.apache.druid.data.input.impl.RegexInputFormat; +import org.apache.druid.data.input.impl.RegexParseSpec; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; @@ -47,6 +59,7 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.autoscaling.ScalingStats; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.EntryExistsException; @@ -222,6 +235,50 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest return testUtils.getIndexMergerV9Factory(); } + /** + * Converts ParseSpec to InputFormat for indexing tests. To be used until {@link FirehoseFactory} + * & {@link InputRowParser} is deprecated and removed. + * + * @param parseSpec + * @return + */ + public static InputFormat createInputFormatFromParseSpec(ParseSpec parseSpec) + { + if (parseSpec instanceof JSONParseSpec) { + JSONParseSpec jsonParseSpec = (JSONParseSpec) parseSpec; + return new JsonInputFormat(jsonParseSpec.getFlattenSpec(), jsonParseSpec.getFeatureSpec(), jsonParseSpec.getKeepNullColumns(), null, null); + } else if (parseSpec instanceof CSVParseSpec) { + CSVParseSpec csvParseSpec = (CSVParseSpec) parseSpec; + boolean getColumnsFromHeader = csvParseSpec.isHasHeaderRow() && csvParseSpec.getSkipHeaderRows() == 0; + return new CsvInputFormat( + csvParseSpec.getColumns(), + csvParseSpec.getListDelimiter(), + getColumnsFromHeader ? null : true, + getColumnsFromHeader ? true : null, + csvParseSpec.getSkipHeaderRows() + ); + } else if (parseSpec instanceof DelimitedParseSpec) { + DelimitedParseSpec delimitedParseSpec = (DelimitedParseSpec) parseSpec; + boolean getColumnsFromHeader = delimitedParseSpec.isHasHeaderRow() && delimitedParseSpec.getSkipHeaderRows() == 0; + return new DelimitedInputFormat( + delimitedParseSpec.getColumns(), + delimitedParseSpec.getListDelimiter(), + delimitedParseSpec.getDelimiter(), + getColumnsFromHeader ? null : true, + getColumnsFromHeader ? true : null, + delimitedParseSpec.getSkipHeaderRows() + ); + } else if (parseSpec instanceof RegexParseSpec) { + RegexParseSpec regexParseSpec = (RegexParseSpec) parseSpec; + return new RegexInputFormat( + regexParseSpec.getPattern(), + regexParseSpec.getListDelimiter(), + regexParseSpec.getColumns()); + } else { + throw new RE(StringUtils.format("Unsupported ParseSpec format %s", parseSpec.toString())); + } + } + public class TestLocalTaskActionClientFactory implements TaskActionClientFactory { @Override diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java index aecd9057506..7bc6548dc20 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java @@ -24,7 +24,10 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.Firehose; +import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.TimestampSpec; @@ -43,13 +46,11 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.FireDepartment; -import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.timeline.partition.NoneShardSpec; import org.hamcrest.CoreMatchers; @@ -64,7 +65,6 @@ import java.io.File; public class TaskSerdeTest { private final ObjectMapper jsonMapper; - private final RowIngestionMetersFactory rowIngestionMetersFactory; private final IndexSpec indexSpec = new IndexSpec(); @Rule @@ -74,14 +74,14 @@ public class TaskSerdeTest { TestUtils testUtils = new TestUtils(); jsonMapper = testUtils.getTestObjectMapper(); - rowIngestionMetersFactory = testUtils.getRowIngestionMetersFactory(); for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) { jsonMapper.registerModule(jacksonModule); } jsonMapper.registerSubtypes( new NamedType(ParallelIndexTuningConfig.class, "index_parallel"), - new NamedType(IndexTuningConfig.class, "index") + new NamedType(IndexTuningConfig.class, "index"), + new NamedType(MockFirehoseFactory.class, "mock") ); } @@ -402,7 +402,7 @@ public class TaskSerdeTest jsonMapper ), new RealtimeIOConfig( - new LocalFirehoseFactory(new File("lol"), "rofl", null), + new MockFirehoseFactory(), (schema, config, metrics) -> null ), @@ -588,6 +588,15 @@ public class TaskSerdeTest Assert.assertEquals("blah", task2.getClasspathPrefix()); } + private static class MockFirehoseFactory implements FirehoseFactory + { + @Override + public Firehose connect(InputRowParser parser, File temporaryDirectory) + { + return null; + } + } + @Test public void testHadoopIndexTaskWithContextSerde() throws Exception { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java index 38f70408232..ef62971e9a2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java @@ -25,7 +25,6 @@ import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.ParseSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; @@ -59,7 +58,6 @@ import org.apache.druid.segment.loading.SegmentLoader; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.segment.loading.SegmentLocalCacheLoader; import org.apache.druid.segment.loading.TombstoneLoadSpec; -import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; import org.junit.Assert; @@ -241,24 +239,22 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn tuningConfig ); } else { - Preconditions.checkArgument(inputFormat == null); + Preconditions.checkArgument(inputFormat == null && parseSpec != null); ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - new LocalFirehoseFactory(inputDirectory, filter, null), + null, + new LocalInputSource(inputDirectory, filter), + createInputFormatFromParseSpec(parseSpec), appendToExisting, dropExisting ); - //noinspection unchecked ingestionSpec = new ParallelIndexIngestionSpec( new DataSchema( - "dataSource", - getObjectMapper().convertValue( - new StringInputRowParser(parseSpec, null), - Map.class - ), + DATASOURCE, + parseSpec.getTimestampSpec(), + parseSpec.getDimensionsSpec(), DEFAULT_METRICS_SPEC, granularitySpec, - null, - getObjectMapper() + null ), ioConfig, tuningConfig diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index fc8d74c1580..109a99fb36e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -903,8 +903,10 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase Map expectedPayload = (Map) expectedReports.get("payload"); Map actualPayload = (Map) actualReports.get("payload"); Assert.assertEquals(expectedPayload.get("ingestionState"), actualPayload.get("ingestionState")); - Assert.assertEquals(expectedPayload.get("rowStats"), actualPayload.get("rowStats")); - Assert.assertEquals(expectedPayload.get("ingestionState"), actualPayload.get("ingestionState")); + + Map expectedTotals = (Map) expectedPayload.get("totals"); + Map actualTotals = (Map) actualReports.get("totals"); + Assert.assertEquals(expectedTotals, actualTotals); List expectedParseExceptionReports = (List) ((Map) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java index 536844f226a..39eaa4af62a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java @@ -26,7 +26,6 @@ import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.ParseSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; @@ -47,7 +46,6 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -267,23 +265,23 @@ public class HashPartitionTaskKillTest extends AbstractMultiPhaseParallelIndexin } else { Preconditions.checkArgument(inputFormat == null); ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - new LocalFirehoseFactory(inputDir, filter, null), - appendToExisting + null, + new LocalInputSource(inputDir, filter), + createInputFormatFromParseSpec(parseSpec), + appendToExisting, + null ); //noinspection unchecked ingestionSpec = new ParallelIndexIngestionSpec( new DataSchema( "dataSource", - getObjectMapper().convertValue( - new StringInputRowParser(parseSpec, null), - Map.class - ), + parseSpec.getTimestampSpec(), + parseSpec.getDimensionsSpec(), new AggregatorFactory[]{ new LongSumAggregatorFactory("val", "val") }, granularitySpec, - null, - getObjectMapper() + null ), ioConfig, tuningConfig diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java index 7d8d681c4ab..6c83c225fa9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java @@ -20,7 +20,6 @@ package org.apache.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.jsontype.NamedType; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -38,7 +37,6 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; import org.hamcrest.CoreMatchers; import org.joda.time.Interval; import org.junit.Assert; @@ -67,9 +65,6 @@ public class ParallelIndexSupervisorTaskSerdeTest { TestUtils testUtils = new TestUtils(); ObjectMapper objectMapper = testUtils.getTestObjectMapper(); - objectMapper.registerSubtypes( - new NamedType(LocalFirehoseFactory.class, "local") - ); return objectMapper; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java index c91c9772e12..71a91dd4f5f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java @@ -27,11 +27,15 @@ import org.apache.commons.codec.Charsets; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InlineInputSource; +import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; import org.apache.druid.rpc.HttpResponseException; @@ -291,6 +295,83 @@ public class ParallelIndexSupervisorTaskTest null ); } + + @Test + public void testFailToConstructWhenBothInputSourceAndParserAreSet() + { + final ObjectMapper mapper = new DefaultObjectMapper(); + final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( + null, + new InlineInputSource("test"), + null, + false, + null + ); + final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( + null, + null, + null, + 10, + 1000L, + null, + null, + null, + null, + new HashedPartitionsSpec(null, 10, null), + new IndexSpec( + new RoaringBitmapSerdeFactory(true), + CompressionStrategy.UNCOMPRESSED, + CompressionStrategy.LZF, + LongEncodingStrategy.LONGS + ), + new IndexSpec(), + 1, + true, + true, + 10000L, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), + null, + 10, + 100, + 20L, + new Duration(3600), + 128, + null, + null, + false, + null, + null, + null, + null, + null + ); + + expectedException.expect(IAE.class); + expectedException.expectMessage("Cannot use parser and inputSource together. Try using inputFormat instead of parser."); + new ParallelIndexIngestionSpec( + new DataSchema( + "datasource", + mapper.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec(null, null, null), + DimensionsSpec.EMPTY, + null, + null, + null + ) + ), + Map.class + ), + null, + null, + null, + mapper + ), + ioConfig, + tuningConfig + ); + } } public static class StaticUtilsTest diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index ddf26dd072c..0b5fbf2dce5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -27,7 +27,6 @@ import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskToolbox; @@ -47,7 +46,6 @@ import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.Partitions; import org.apache.druid.timeline.SegmentTimeline; @@ -928,27 +926,22 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv ingestionSpec = new ParallelIndexIngestionSpec( new DataSchema( "dataSource", - getObjectMapper().convertValue( - new StringInputRowParser( - DEFAULT_PARSE_SPEC, - null - ), - Map.class - ), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, + DEFAULT_TIMESTAMP_SPEC, + DEFAULT_DIMENSIONS_SPEC, + DEFAULT_METRICS_SPEC, new UniformGranularitySpec( segmentGranularity, Granularities.MINUTE, interval == null ? null : Collections.singletonList(interval) ), - null, - getObjectMapper() + null ), new ParallelIndexIOConfig( - new LocalFirehoseFactory(inputDir, inputSourceFilter, null), - appendToExisting + null, + new LocalInputSource(inputDir, inputSourceFilter), + createInputFormatFromParseSpec(DEFAULT_PARSE_SPEC), + appendToExisting, + null ), tuningConfig ); @@ -966,10 +959,10 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv private String getErrorMessageForUnparseableTimestamp() { - return useInputFormatApi ? StringUtils.format( + return StringUtils.format( "Timestamp[2017unparseable] is unparseable! Event: {ts=2017unparseable} (Path: %s, Record: 5, Line: 5)", new File(inputDir, "test_0").toURI() - ) : "Timestamp[2017unparseable] is unparseable! Event: {ts=2017unparseable}"; + ); } private static class SettableSplittableLocalInputSource extends LocalInputSource diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java deleted file mode 100644 index 591a3adb84c..00000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ /dev/null @@ -1,615 +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.druid.indexing.firehose; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.inject.Binder; -import com.google.inject.Module; -import org.apache.druid.client.coordinator.CoordinatorClient; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.GuiceAnnotationIntrospector; -import org.apache.druid.guice.GuiceInjectableValues; -import org.apache.druid.guice.GuiceInjectors; -import org.apache.druid.indexing.common.ReingestionTimelineUtils; -import org.apache.druid.indexing.common.RetryPolicyConfig; -import org.apache.druid.indexing.common.RetryPolicyFactory; -import org.apache.druid.indexing.common.SegmentCacheManagerFactory; -import org.apache.druid.indexing.common.TestUtils; -import org.apache.druid.indexing.common.config.TaskStorageConfig; -import org.apache.druid.indexing.common.task.NoopTask; -import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; -import org.apache.druid.indexing.overlord.Segments; -import org.apache.druid.indexing.overlord.TaskLockbox; -import org.apache.druid.indexing.overlord.TaskStorage; -import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.JodaUtils; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; -import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.query.expression.TestExprMacroTable; -import org.apache.druid.query.filter.SelectorDimFilter; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexMergerV9; -import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; -import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; -import org.apache.druid.segment.loading.LocalDataSegmentPuller; -import org.apache.druid.segment.loading.LocalLoadSpec; -import org.apache.druid.segment.realtime.firehose.CombiningFirehoseFactory; -import org.apache.druid.segment.transform.ExpressionTransform; -import org.apache.druid.segment.transform.TransformSpec; -import org.apache.druid.server.metrics.NoopServiceEmitter; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.TimelineObjectHolder; -import org.apache.druid.timeline.partition.NumberedPartitionChunk; -import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.apache.druid.timeline.partition.PartitionChunk; -import org.apache.druid.timeline.partition.PartitionHolder; -import org.easymock.EasyMock; -import org.joda.time.Interval; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -/** - * - */ -@RunWith(Parameterized.class) -public class IngestSegmentFirehoseFactoryTest -{ - private static final ObjectMapper MAPPER; - private static final IndexMergerV9 INDEX_MERGER_V9; - private static final IndexIO INDEX_IO; - private static final TaskStorage TASK_STORAGE; - private static final IndexerSQLMetadataStorageCoordinator MDC; - private static final TaskLockbox TASK_LOCKBOX; - private static final Task TASK; - - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); - - static { - TestUtils testUtils = new TestUtils(); - MAPPER = setupInjectablesInObjectMapper(TestHelper.makeJsonMapper()); - INDEX_MERGER_V9 = testUtils.getTestIndexMergerV9(); - INDEX_IO = testUtils.getTestIndexIO(); - TASK_STORAGE = new HeapMemoryTaskStorage( - new TaskStorageConfig(null) - { - } - ); - MDC = new IndexerSQLMetadataStorageCoordinator(null, null, null) - { - private final Set published = new HashSet<>(); - - @Override - public List retrieveUsedSegmentsForIntervals( - String dataSource, - List interval, - Segments visibility - ) - { - return ImmutableList.copyOf(SEGMENT_SET); - } - - @Override - public List retrieveUnusedSegmentsForInterval(String dataSource, Interval interval) - { - return ImmutableList.of(); - } - - @Override - public Set announceHistoricalSegments(Set segments) - { - Set added = new HashSet<>(); - for (final DataSegment segment : segments) { - if (published.add(segment)) { - added.add(segment); - } - } - - return ImmutableSet.copyOf(added); - } - - @Override - public void deleteSegments(Set segments) - { - // do nothing - } - }; - TASK_LOCKBOX = new TaskLockbox(TASK_STORAGE, MDC); - TASK = NoopTask.create(); - TASK_LOCKBOX.add(TASK); - } - - @Parameterized.Parameters(name = "{0}") - public static Collection constructorFeeder() throws IOException - { - final IndexSpec indexSpec = new IndexSpec(); - - final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() - .withMinTimestamp(JodaUtils.MIN_INSTANT) - .withDimensionsSpec(ROW_PARSER) - .withMetrics( - new LongSumAggregatorFactory(METRIC_LONG_NAME, DIM_LONG_NAME), - new DoubleSumAggregatorFactory(METRIC_FLOAT_NAME, DIM_FLOAT_NAME) - ) - .build(); - final IncrementalIndex index = new OnheapIncrementalIndex.Builder() - .setIndexSchema(schema) - .setMaxRowCount(MAX_ROWS * MAX_SHARD_NUMBER) - .build(); - - for (Integer i = 0; i < MAX_ROWS; ++i) { - index.add(ROW_PARSER.parseBatch(buildRow(i.longValue())).get(0)); - } - - FileUtils.mkdirp(PERSIST_DIR); - INDEX_MERGER_V9.persist(index, PERSIST_DIR, indexSpec, null); - - final CoordinatorClient cc = new CoordinatorClient(null, null) - { - @Override - public Collection fetchUsedSegmentsInDataSourceForIntervals( - String dataSource, - List intervals - ) - { - return ImmutableSet.copyOf(SEGMENT_SET); - } - }; - - SegmentHandoffNotifierFactory notifierFactory = EasyMock.createNiceMock(SegmentHandoffNotifierFactory.class); - EasyMock.replay(notifierFactory); - - final SegmentCacheManagerFactory slf = new SegmentCacheManagerFactory(MAPPER); - final RetryPolicyFactory retryPolicyFactory = new RetryPolicyFactory(new RetryPolicyConfig()); - - Collection values = new ArrayList<>(); - for (InputRowParser parser : Arrays.asList( - ROW_PARSER, - new MapInputRowParser( - new JSONParseSpec( - new TimestampSpec(TIME_COLUMN, "auto", null), - DimensionsSpec.builder() - .setDimensionExclusions(ImmutableList.of(DIM_FLOAT_NAME, DIM_LONG_NAME)) - .build(), - null, - null, - null - ) - ) - )) { - for (List dim_names : Arrays.>asList(null, ImmutableList.of(DIM_NAME))) { - for (List metric_names : Arrays.>asList( - null, - ImmutableList.of(METRIC_LONG_NAME, METRIC_FLOAT_NAME) - )) { - for (Boolean wrapInCombining : Arrays.asList(false, true)) { - final IngestSegmentFirehoseFactory isfFactory = new IngestSegmentFirehoseFactory( - TASK.getDataSource(), - Intervals.ETERNITY, - null, - new SelectorDimFilter(DIM_NAME, DIM_VALUE, null), - dim_names, - metric_names, - null, - INDEX_IO, - cc, - slf, - retryPolicyFactory - ); - final FirehoseFactory factory = wrapInCombining - ? new CombiningFirehoseFactory(ImmutableList.of(isfFactory)) - : isfFactory; - values.add( - new Object[]{ - StringUtils.format( - "DimNames[%s]MetricNames[%s]ParserDimNames[%s]WrapInCombining[%s]", - dim_names == null ? "null" : "dims", - metric_names == null ? "null" : "metrics", - parser == ROW_PARSER ? "dims" : "null", - wrapInCombining - ), - factory, - parser - } - ); - } - } - } - } - return values; - } - - public static ObjectMapper setupInjectablesInObjectMapper(ObjectMapper objectMapper) - { - objectMapper.registerModule( - new SimpleModule("testModule").registerSubtypes(LocalLoadSpec.class) - ); - - final GuiceAnnotationIntrospector guiceIntrospector = new GuiceAnnotationIntrospector(); - objectMapper.setAnnotationIntrospectors( - new AnnotationIntrospectorPair( - guiceIntrospector, - objectMapper.getSerializationConfig().getAnnotationIntrospector() - ), - new AnnotationIntrospectorPair( - guiceIntrospector, - objectMapper.getDeserializationConfig().getAnnotationIntrospector() - ) - ); - objectMapper.setInjectableValues( - new GuiceInjectableValues( - GuiceInjectors.makeStartupInjectorWithModules( - ImmutableList.of( - new Module() - { - @Override - public void configure(Binder binder) - { - binder.bind(LocalDataSegmentPuller.class); - binder.bind(ExprMacroTable.class).toInstance(TestExprMacroTable.INSTANCE); - } - } - ) - ) - ) - ); - return objectMapper; - } - - public IngestSegmentFirehoseFactoryTest( - String testName, - FirehoseFactory factory, - InputRowParser rowParser - ) - { - this.factory = factory; - - // Must decorate the parser, since IngestSegmentFirehoseFactory will undecorate it. - this.rowParser = TransformSpec.NONE.decorate(rowParser); - } - - private static final Logger log = new Logger(IngestSegmentFirehoseFactoryTest.class); - private static final String DATA_SOURCE_NAME = "testDataSource"; - private static final String DATA_SOURCE_VERSION = "version"; - private static final Integer BINARY_VERSION = -1; - private static final String DIM_NAME = "testDimName"; - private static final String DIM_VALUE = "testDimValue"; - private static final String DIM_LONG_NAME = "testDimLongName"; - private static final String DIM_FLOAT_NAME = "testDimFloatName"; - private static final String METRIC_LONG_NAME = "testLongMetric"; - private static final String METRIC_FLOAT_NAME = "testFloatMetric"; - private static final Long METRIC_LONG_VALUE = 1L; - private static final Float METRIC_FLOAT_VALUE = 1.0f; - private static final String TIME_COLUMN = "ts"; - private static final Integer MAX_SHARD_NUMBER = 10; - private static final Integer MAX_ROWS = 10; - private static final File TMP_DIR = FileUtils.createTempDir(); - private static final File PERSIST_DIR = Paths.get(TMP_DIR.getAbsolutePath(), "indexTestMerger").toFile(); - private static final List SEGMENT_SET = new ArrayList<>(MAX_SHARD_NUMBER); - - private final FirehoseFactory factory; - private final InputRowParser rowParser; - private File tempDir; - - private static final InputRowParser> ROW_PARSER = new MapInputRowParser( - new TimeAndDimsParseSpec( - new TimestampSpec(TIME_COLUMN, "auto", null), - DimensionsSpec.builder() - .setDimensions(DimensionsSpec.getDefaultSchemas(ImmutableList.of(DIM_NAME))) - .setDimensionExclusions(ImmutableList.of(DIM_FLOAT_NAME, DIM_LONG_NAME)) - .build() - ) - ); - - private static Map buildRow(Long ts) - { - return ImmutableMap.of( - TIME_COLUMN, ts, - DIM_NAME, DIM_VALUE, - DIM_FLOAT_NAME, METRIC_FLOAT_VALUE, - DIM_LONG_NAME, METRIC_LONG_VALUE - ); - } - - private static DataSegment buildSegment(Integer shardNumber) - { - Preconditions.checkArgument(shardNumber < MAX_SHARD_NUMBER); - Preconditions.checkArgument(shardNumber >= 0); - return new DataSegment( - DATA_SOURCE_NAME, - Intervals.ETERNITY, - DATA_SOURCE_VERSION, - ImmutableMap.of( - "type", "local", - "path", PERSIST_DIR.getAbsolutePath() - ), - ImmutableList.of(DIM_NAME), - ImmutableList.of(METRIC_LONG_NAME, METRIC_FLOAT_NAME), - new NumberedShardSpec( - shardNumber, - MAX_SHARD_NUMBER - ), - BINARY_VERSION, - 0L - ); - } - - @BeforeClass - public static void setUpStatic() - { - for (int i = 0; i < MAX_SHARD_NUMBER; ++i) { - SEGMENT_SET.add(buildSegment(i)); - } - } - - @AfterClass - public static void tearDownStatic() - { - recursivelyDelete(TMP_DIR); - } - - private static void recursivelyDelete(final File dir) - { - if (dir != null) { - if (dir.isDirectory()) { - final File[] files = dir.listFiles(); - if (files != null) { - for (File file : files) { - recursivelyDelete(file); - } - } - } else { - if (!dir.delete()) { - log.warn("Could not delete file at [%s]", dir.getAbsolutePath()); - } - } - } - } - - @Before - public void setup() throws IOException - { - tempDir = temporaryFolder.newFolder(); - } - - @After - public void teardown() - { - tempDir.delete(); - } - - @Test - public void sanityTest() - { - if (factory instanceof CombiningFirehoseFactory) { - // This method tests IngestSegmentFirehoseFactory-specific methods. - return; - } - final IngestSegmentFirehoseFactory isfFactory = (IngestSegmentFirehoseFactory) factory; - Assert.assertEquals(TASK.getDataSource(), isfFactory.getDataSource()); - if (isfFactory.getDimensions() != null) { - Assert.assertArrayEquals(new String[]{DIM_NAME}, isfFactory.getDimensions().toArray()); - } - Assert.assertEquals(Intervals.ETERNITY, isfFactory.getInterval()); - if (isfFactory.getMetrics() != null) { - Assert.assertEquals( - ImmutableSet.of(METRIC_LONG_NAME, METRIC_FLOAT_NAME), - ImmutableSet.copyOf(isfFactory.getMetrics()) - ); - } - } - - @Test - public void simpleFirehoseReadingTest() throws IOException - { - Assert.assertEquals(MAX_SHARD_NUMBER.longValue(), SEGMENT_SET.size()); - Integer rowcount = 0; - try (final Firehose firehose = factory.connect(rowParser, TMP_DIR)) { - while (firehose.hasMore()) { - InputRow row = firehose.nextRow(); - Assert.assertArrayEquals(new String[]{DIM_NAME}, row.getDimensions().toArray()); - Assert.assertArrayEquals(new String[]{DIM_VALUE}, row.getDimension(DIM_NAME).toArray()); - Assert.assertEquals(METRIC_LONG_VALUE.longValue(), row.getMetric(METRIC_LONG_NAME)); - Assert.assertEquals( - METRIC_FLOAT_VALUE, - row.getMetric(METRIC_FLOAT_NAME).floatValue(), - METRIC_FLOAT_VALUE * 0.0001 - ); - ++rowcount; - } - } - Assert.assertEquals((int) MAX_SHARD_NUMBER * MAX_ROWS, (int) rowcount); - } - - @Test - public void testTransformSpec() throws IOException - { - Assert.assertEquals(MAX_SHARD_NUMBER.longValue(), SEGMENT_SET.size()); - Integer rowcount = 0; - final TransformSpec transformSpec = new TransformSpec( - new SelectorDimFilter(ColumnHolder.TIME_COLUMN_NAME, "1", null), - ImmutableList.of( - new ExpressionTransform(METRIC_FLOAT_NAME, METRIC_FLOAT_NAME + " * 10", ExprMacroTable.nil()) - ) - ); - int skipped = 0; - try (final Firehose firehose = - factory.connect(transformSpec.decorate(rowParser), TMP_DIR)) { - while (firehose.hasMore()) { - InputRow row = firehose.nextRow(); - if (row == null) { - skipped++; - continue; - } - Assert.assertArrayEquals(new String[]{DIM_NAME}, row.getDimensions().toArray()); - Assert.assertArrayEquals(new String[]{DIM_VALUE}, row.getDimension(DIM_NAME).toArray()); - Assert.assertEquals(METRIC_LONG_VALUE.longValue(), row.getMetric(METRIC_LONG_NAME).longValue()); - Assert.assertEquals( - METRIC_FLOAT_VALUE * 10, - row.getMetric(METRIC_FLOAT_NAME).floatValue(), - METRIC_FLOAT_VALUE * 0.0001 - ); - ++rowcount; - } - } - Assert.assertEquals(90, skipped); - Assert.assertEquals((int) MAX_ROWS, (int) rowcount); - } - - @Test - public void testGetUniqueDimensionsAndMetrics() - { - final int numSegmentsPerPartitionChunk = 5; - final int numPartitionChunksPerTimelineObject = 10; - final int numSegments = numSegmentsPerPartitionChunk * numPartitionChunksPerTimelineObject; - final Interval interval = Intervals.of("2017-01-01/2017-01-02"); - final String version = "1"; - - final List> timelineSegments = new ArrayList<>(); - for (int i = 0; i < numPartitionChunksPerTimelineObject; i++) { - final List> chunks = new ArrayList<>(); - for (int j = 0; j < numSegmentsPerPartitionChunk; j++) { - final List dims = IntStream.range(i, i + numSegmentsPerPartitionChunk) - .mapToObj(suffix -> "dim" + suffix) - .collect(Collectors.toList()); - final List metrics = IntStream.range(i, i + numSegmentsPerPartitionChunk) - .mapToObj(suffix -> "met" + suffix) - .collect(Collectors.toList()); - final DataSegment segment = new DataSegment( - "ds", - interval, - version, - ImmutableMap.of(), - dims, - metrics, - new NumberedShardSpec(numPartitionChunksPerTimelineObject, i), - 1, - 1 - ); - - final PartitionChunk partitionChunk = new NumberedPartitionChunk<>( - i, - numPartitionChunksPerTimelineObject, - segment - ); - chunks.add(partitionChunk); - } - final TimelineObjectHolder timelineHolder = new TimelineObjectHolder<>( - interval, - version, - new PartitionHolder<>(chunks) - ); - timelineSegments.add(timelineHolder); - } - - final String[] expectedDims = new String[]{ - "dim9", - "dim10", - "dim11", - "dim12", - "dim13", - "dim8", - "dim7", - "dim6", - "dim5", - "dim4", - "dim3", - "dim2", - "dim1", - "dim0" - }; - final String[] expectedMetrics = new String[]{ - "met9", - "met10", - "met11", - "met12", - "met13", - "met8", - "met7", - "met6", - "met5", - "met4", - "met3", - "met2", - "met1", - "met0" - }; - Assert.assertEquals( - Arrays.asList(expectedDims), - ReingestionTimelineUtils.getUniqueDimensions(timelineSegments, null) - ); - Assert.assertEquals( - Arrays.asList(expectedMetrics), - ReingestionTimelineUtils.getUniqueMetrics(timelineSegments) - ); - } - - private static ServiceEmitter newMockEmitter() - { - return new NoopServiceEmitter(); - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java deleted file mode 100644 index 61527089728..00000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ /dev/null @@ -1,453 +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.druid.indexing.firehose; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; -import org.apache.druid.client.coordinator.CoordinatorClient; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.indexing.common.RetryPolicyConfig; -import org.apache.druid.indexing.common.RetryPolicyFactory; -import org.apache.druid.indexing.common.SegmentCacheManagerFactory; -import org.apache.druid.indexing.common.TestUtils; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.JodaUtils; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.query.filter.TrueDimFilter; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexMergerV9; -import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; -import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IndexSizeExceededException; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; -import org.apache.druid.segment.transform.TransformSpec; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.partition.LinearShardSpec; -import org.easymock.EasyMock; -import org.joda.time.Interval; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import java.util.stream.Collectors; - -@RunWith(Parameterized.class) -public class IngestSegmentFirehoseFactoryTimelineTest -{ - private static final String DATA_SOURCE = "foo"; - private static final String TIME_COLUMN = "t"; - private static final String[] DIMENSIONS = new String[]{"d1"}; - private static final String[] METRICS = new String[]{"m1"}; - - // Must decorate the parser, since IngestSegmentFirehoseFactory will undecorate it. - private static final InputRowParser> ROW_PARSER = TransformSpec.NONE.decorate( - new MapInputRowParser( - new JSONParseSpec( - new TimestampSpec(TIME_COLUMN, "auto", null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(Arrays.asList(DIMENSIONS)) - ), - null, - null, - null - ) - ) - ); - - private final IngestSegmentFirehoseFactory factory; - private final File tmpDir; - private final int expectedCount; - private final long expectedSum; - private final int segmentCount; - - private static final ObjectMapper MAPPER; - private static final IndexIO INDEX_IO; - private static final IndexMergerV9 INDEX_MERGER_V9; - - static { - TestUtils testUtils = new TestUtils(); - MAPPER = IngestSegmentFirehoseFactoryTest.setupInjectablesInObjectMapper(testUtils.getTestObjectMapper()); - INDEX_IO = testUtils.getTestIndexIO(); - INDEX_MERGER_V9 = testUtils.getTestIndexMergerV9(); - } - - public IngestSegmentFirehoseFactoryTimelineTest( - String name, - IngestSegmentFirehoseFactory factory, - File tmpDir, - int expectedCount, - long expectedSum, - int segmentCount - ) - { - this.factory = factory; - this.tmpDir = tmpDir; - this.expectedCount = expectedCount; - this.expectedSum = expectedSum; - this.segmentCount = segmentCount; - } - - @Test - public void test() throws Exception - { - // Junit 4.12 doesn't have a good way to run tearDown after multiple tests in a Parameterized - // class run. (Junit 4.13 adds @AfterParam but isn't released yet.) Fake it by just running - // "tests" in series inside one @Test. - testSimple(); - testSplit(); - } - - private void testSimple() throws Exception - { - int count = 0; - long sum = 0; - - try (final Firehose firehose = factory.connect(ROW_PARSER, tmpDir)) { - while (firehose.hasMore()) { - final InputRow row = firehose.nextRow(); - count++; - sum += row.getMetric(METRICS[0]).longValue(); - } - } - - Assert.assertEquals("count", expectedCount, count); - Assert.assertEquals("sum", expectedSum, sum); - } - - private void testSplit() throws Exception - { - Assert.assertTrue(factory.isSplittable()); - final int numSplits = factory.getNumSplits(null); - // We set maxInputSegmentBytesPerSplit to 2 so each segment should become a byte. - Assert.assertEquals(segmentCount, numSplits); - final List>> splits = - factory.getSplits(null).collect(Collectors.toList()); - Assert.assertEquals(numSplits, splits.size()); - - int count = 0; - long sum = 0; - - for (InputSplit> split : splits) { - final FiniteFirehoseFactory> splitFactory = - factory.withSplit(split); - try (final Firehose firehose = splitFactory.connect(ROW_PARSER, tmpDir)) { - while (firehose.hasMore()) { - final InputRow row = firehose.nextRow(); - count++; - sum += row.getMetric(METRICS[0]).longValue(); - } - } - } - - Assert.assertEquals("count", expectedCount, count); - Assert.assertEquals("sum", expectedSum, sum); - - } - - @After - public void tearDown() throws Exception - { - FileUtils.deleteDirectory(tmpDir); - } - - private static TestCase tc( - String intervalString, - int expectedCount, - long expectedSum, - DataSegmentMaker... segmentMakers - ) - { - final File tmpDir = FileUtils.createTempDir(); - final Set segments = new HashSet<>(); - for (DataSegmentMaker segmentMaker : segmentMakers) { - segments.add(segmentMaker.make(tmpDir)); - } - - return new TestCase( - tmpDir, - Intervals.of(intervalString), - expectedCount, - expectedSum, - segments - ); - } - - private static DataSegmentMaker ds( - String intervalString, - String version, - int partitionNum, - InputRow... rows - ) - { - return new DataSegmentMaker(Intervals.of(intervalString), version, partitionNum, Arrays.asList(rows)); - } - - private static InputRow ir(String timeString, long metricValue) - { - return new MapBasedInputRow( - DateTimes.of(timeString).getMillis(), - Arrays.asList(DIMENSIONS), - ImmutableMap.of( - TIME_COLUMN, DateTimes.of(timeString).toString(), - DIMENSIONS[0], "bar", - METRICS[0], metricValue - ) - ); - } - - private static Map persist(File tmpDir, InputRow... rows) - { - final File persistDir = new File(tmpDir, UUID.randomUUID().toString()); - final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() - .withMinTimestamp(JodaUtils.MIN_INSTANT) - .withDimensionsSpec(ROW_PARSER) - .withMetrics(new LongSumAggregatorFactory(METRICS[0], METRICS[0])) - .build(); - final IncrementalIndex index = new OnheapIncrementalIndex.Builder() - .setIndexSchema(schema) - .setMaxRowCount(rows.length) - .build(); - - for (InputRow row : rows) { - try { - index.add(row); - } - catch (IndexSizeExceededException e) { - throw new RuntimeException(e); - } - } - - try { - INDEX_MERGER_V9.persist(index, persistDir, new IndexSpec(), null); - } - catch (IOException e) { - throw new RuntimeException(e); - } - - return ImmutableMap.of( - "type", "local", - "path", persistDir.getAbsolutePath() - ); - } - - @Parameterized.Parameters(name = "{0}") - public static Collection constructorFeeder() - { - final List testCases = ImmutableList.of( - tc( - "2000/2000T02", 3, 7, - ds("2000/2000T01", "v1", 0, ir("2000", 1), ir("2000T00:01", 2)), - ds("2000T01/2000T02", "v1", 0, ir("2000T01", 4)) - ) /* Adjacent segments */, - tc( - "2000/2000T02", 3, 7, - ds("2000/2000T02", "v1", 0, ir("2000", 1), ir("2000T00:01", 2), ir("2000T01", 8)), - ds("2000T01/2000T02", "v2", 0, ir("2000T01:01", 4)) - ) /* 1H segment overlaid on top of 2H segment */, - tc( - "2000/2000-01-02", 4, 23, - ds("2000/2000-01-02", "v1", 0, ir("2000", 1), ir("2000T00:01", 2), ir("2000T01", 8), ir("2000T02", 16)), - ds("2000T01/2000T02", "v2", 0, ir("2000T01:01", 4)) - ) /* 1H segment overlaid on top of 1D segment */, - tc( - "2000/2000T02", 4, 15, - ds("2000/2000T02", "v1", 0, ir("2000", 1), ir("2000T00:01", 2), ir("2000T01", 8)), - ds("2000/2000T02", "v1", 1, ir("2000T01:01", 4)) - ) /* Segment set with two segments for the same interval */, - tc( - "2000T01/2000T02", 1, 2, - ds("2000/2000T03", "v1", 0, ir("2000", 1), ir("2000T01", 2), ir("2000T02", 4)) - ) /* Segment wider than desired interval */, - tc( - "2000T02/2000T04", 2, 12, - ds("2000/2000T03", "v1", 0, ir("2000", 1), ir("2000T01", 2), ir("2000T02", 4)), - ds("2000T03/2000T04", "v1", 0, ir("2000T03", 8)) - ) /* Segment intersecting desired interval */ - ); - - final List constructors = new ArrayList<>(); - - for (final TestCase testCase : testCases) { - SegmentHandoffNotifierFactory notifierFactory = EasyMock.createNiceMock(SegmentHandoffNotifierFactory.class); - EasyMock.replay(notifierFactory); - final SegmentCacheManagerFactory slf = new SegmentCacheManagerFactory(MAPPER); - final RetryPolicyFactory retryPolicyFactory = new RetryPolicyFactory(new RetryPolicyConfig()); - final CoordinatorClient cc = new CoordinatorClient(null, null) - { - @Override - public Collection fetchUsedSegmentsInDataSourceForIntervals( - String dataSource, - List intervals - ) - { - // Expect the interval we asked for - if (intervals.equals(ImmutableList.of(testCase.interval))) { - return ImmutableSet.copyOf(testCase.segments); - } else { - throw new IllegalArgumentException("BAD"); - } - } - - @Override - public DataSegment fetchUsedSegment(String dataSource, String segmentId) - { - return testCase.segments - .stream() - .filter(s -> s.getId().toString().equals(segmentId)) - .findAny() - .get(); // throwing if not found is exactly what the real code does - } - }; - final IngestSegmentFirehoseFactory factory = new IngestSegmentFirehoseFactory( - DATA_SOURCE, - testCase.interval, - null, - TrueDimFilter.instance(), - Arrays.asList(DIMENSIONS), - Arrays.asList(METRICS), - // Split as much as possible - 1L, - INDEX_IO, - cc, - slf, - retryPolicyFactory - ); - - constructors.add( - new Object[]{ - testCase.toString(), - factory, - testCase.tmpDir, - testCase.expectedCount, - testCase.expectedSum, - testCase.segments.size() - } - ); - } - - return constructors; - } - - private static class TestCase - { - final File tmpDir; - final Interval interval; - final int expectedCount; - final long expectedSum; - final Set segments; - - public TestCase( - File tmpDir, - Interval interval, - int expectedCount, - long expectedSum, - Set segments - ) - { - this.tmpDir = tmpDir; - this.interval = interval; - this.expectedCount = expectedCount; - this.expectedSum = expectedSum; - this.segments = segments; - } - - @Override - public String toString() - { - final List segmentIds = new ArrayList<>(); - for (DataSegment segment : segments) { - segmentIds.add(segment.getId()); - } - return "TestCase{" + - "interval=" + interval + - ", expectedCount=" + expectedCount + - ", expectedSum=" + expectedSum + - ", segments=" + segmentIds + - '}'; - } - } - - private static class DataSegmentMaker - { - final Interval interval; - final String version; - final int partitionNum; - final List rows; - - public DataSegmentMaker( - Interval interval, - String version, - int partitionNum, - List rows - ) - { - this.interval = interval; - this.version = version; - this.partitionNum = partitionNum; - this.rows = rows; - } - - public DataSegment make(File tmpDir) - { - final Map loadSpec = persist(tmpDir, Iterables.toArray(rows, InputRow.class)); - - return new DataSegment( - DATA_SOURCE, - interval, - version, - loadSpec, - Arrays.asList(DIMENSIONS), - Arrays.asList(METRICS), - new LinearShardSpec(partitionNum), - -1, - 2L - ); - } - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RemoteTaskRunnerTest.java index 796ba351c4a..99b6e39717f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RemoteTaskRunnerTest.java @@ -37,7 +37,7 @@ import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IndexingServiceCondition; -import org.apache.druid.indexing.common.TestRealtimeTask; +import org.apache.druid.indexing.common.TestIndexTask; import org.apache.druid.indexing.common.TestTasks; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.task.Task; @@ -251,7 +251,7 @@ public class RemoteTaskRunnerTest { doSetup(); - TestRealtimeTask task1 = new TestRealtimeTask( + TestIndexTask task1 = new TestIndexTask( "rt1", new TaskResource("rt1", 1), "foo", @@ -262,7 +262,7 @@ public class RemoteTaskRunnerTest Assert.assertTrue(taskAnnounced(task1.getId())); mockWorkerRunningTask(task1); - TestRealtimeTask task2 = new TestRealtimeTask( + TestIndexTask task2 = new TestIndexTask( "rt2", new TaskResource("rt1", 1), "foo", @@ -271,7 +271,7 @@ public class RemoteTaskRunnerTest ); remoteTaskRunner.run(task2); - TestRealtimeTask task3 = new TestRealtimeTask( + TestIndexTask task3 = new TestIndexTask( "rt3", new TaskResource("rt2", 1), "foo", @@ -314,7 +314,7 @@ public class RemoteTaskRunnerTest { doSetup(); - TestRealtimeTask task1 = new TestRealtimeTask( + TestIndexTask task1 = new TestIndexTask( "rt1", new TaskResource("rt1", 1), "foo", @@ -325,7 +325,7 @@ public class RemoteTaskRunnerTest Assert.assertTrue(taskAnnounced(task1.getId())); mockWorkerRunningTask(task1); - TestRealtimeTask task2 = new TestRealtimeTask( + TestIndexTask task2 = new TestIndexTask( "rt2", new TaskResource("rt2", 3), "foo", @@ -334,7 +334,7 @@ public class RemoteTaskRunnerTest ); remoteTaskRunner.run(task2); - TestRealtimeTask task3 = new TestRealtimeTask( + TestIndexTask task3 = new TestIndexTask( "rt3", new TaskResource("rt3", 2), "foo", @@ -406,7 +406,7 @@ public class RemoteTaskRunnerTest makeRemoteTaskRunner(rtrConfig); - TestRealtimeTask task1 = new TestRealtimeTask( + TestIndexTask task1 = new TestIndexTask( "first", new TaskResource("first", 1), "foo", @@ -417,7 +417,7 @@ public class RemoteTaskRunnerTest Assert.assertTrue(taskAnnounced(task1.getId())); mockWorkerRunningTask(task1); - TestRealtimeTask task = new TestRealtimeTask( + TestIndexTask task = new TestIndexTask( "second", new TaskResource("task", 2), "foo", @@ -426,7 +426,7 @@ public class RemoteTaskRunnerTest ); remoteTaskRunner.run(task); - TestRealtimeTask task2 = new TestRealtimeTask( + TestIndexTask task2 = new TestIndexTask( "second", new TaskResource("second", 2), "foo", @@ -457,7 +457,7 @@ public class RemoteTaskRunnerTest public void testRunWithTaskComplete() throws Exception { doSetup(); - TestRealtimeTask task1 = new TestRealtimeTask( + TestIndexTask task1 = new TestIndexTask( "testTask", new TaskResource("testTask", 2), "foo", @@ -817,11 +817,11 @@ public class RemoteTaskRunnerTest makeRemoteTaskRunner(rtrConfig); - TestRealtimeTask task1 = new TestRealtimeTask( - "realtime1", - new TaskResource("realtime1", 1), + TestIndexTask task1 = new TestIndexTask( + "test_index1", + new TaskResource("test_index1", 1), "foo", - TaskStatus.success("realtime1"), + TaskStatus.success("test_index1"), jsonMapper ); Future taskFuture1 = remoteTaskRunner.run(task1); @@ -835,11 +835,11 @@ public class RemoteTaskRunnerTest remoteTaskRunner.findWorkerRunningTask(task1.getId()).getContinuouslyFailedTasksCount() ); - TestRealtimeTask task2 = new TestRealtimeTask( - "realtime2", - new TaskResource("realtime2", 1), + TestIndexTask task2 = new TestIndexTask( + "test_index2", + new TaskResource("test_index2", 1), "foo", - TaskStatus.running("realtime2"), + TaskStatus.running("test_index2"), jsonMapper ); Future taskFuture2 = remoteTaskRunner.run(task2); @@ -870,11 +870,11 @@ public class RemoteTaskRunnerTest remoteTaskRunner.findWorkerRunningTask(task2.getId()).getContinuouslyFailedTasksCount() ); - TestRealtimeTask task3 = new TestRealtimeTask( - "realtime3", - new TaskResource("realtime3", 1), + TestIndexTask task3 = new TestIndexTask( + "test_index3", + new TaskResource("test_index3", 1), "foo", - TaskStatus.running("realtime3"), + TaskStatus.running("test_index3"), jsonMapper ); Future taskFuture3 = remoteTaskRunner.run(task3); @@ -909,7 +909,7 @@ public class RemoteTaskRunnerTest for (int i = 1; i < 13; i++) { String taskId = StringUtils.format("rt-%d", i); - TestRealtimeTask task = new TestRealtimeTask( + TestIndexTask task = new TestIndexTask( taskId, new TaskResource(taskId, 1), "foo", @@ -964,7 +964,7 @@ public class RemoteTaskRunnerTest for (int i = 1; i < 13; i++) { String taskId = StringUtils.format("rt-%d", i); - TestRealtimeTask task = new TestRealtimeTask( + TestIndexTask task = new TestIndexTask( taskId, new TaskResource(taskId, 1), "foo", @@ -1009,14 +1009,14 @@ public class RemoteTaskRunnerTest makeRemoteTaskRunner(rtrConfig); - TestRealtimeTask task1 = new TestRealtimeTask( - "realtime1", new TaskResource("realtime1", 1), "foo", TaskStatus.success("realtime1"), jsonMapper + TestIndexTask task1 = new TestIndexTask( + "test_index1", new TaskResource("test_index1", 1), "foo", TaskStatus.success("test_index1"), jsonMapper ); - TestRealtimeTask task2 = new TestRealtimeTask( - "realtime2", new TaskResource("realtime2", 1), "foo", TaskStatus.success("realtime2"), jsonMapper + TestIndexTask task2 = new TestIndexTask( + "test_index2", new TaskResource("test_index2", 1), "foo", TaskStatus.success("test_index2"), jsonMapper ); - TestRealtimeTask task3 = new TestRealtimeTask( - "realtime3", new TaskResource("realtime3", 1), "foo", TaskStatus.success("realtime3"), jsonMapper + TestIndexTask task3 = new TestIndexTask( + "test_index3", new TaskResource("test_index3", 1), "foo", TaskStatus.success("test_index3"), jsonMapper ); Future taskFuture1 = remoteTaskRunner.run(task1); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java index a80aabdc34c..5f6375e6297 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java @@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableSet; import net.thisptr.jackson.jq.internal.misc.Lists; import org.apache.druid.client.indexing.SamplerResponse; import org.apache.druid.client.indexing.SamplerResponse.SamplerResponseRow; -import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSource; @@ -39,11 +38,9 @@ import org.apache.druid.data.input.impl.InlineInputSource; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.JsonInputFormat; -import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.NestedDataModule; import org.apache.druid.indexing.seekablestream.RecordSupplierInputSource; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; @@ -60,13 +57,9 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.SelectorDimFilter; -import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.nested.StructuredData; -import org.apache.druid.segment.realtime.firehose.InlineFirehoseFactory; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -132,9 +125,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest @Parameterized.Parameters(name = "parserType = {0}, useInputFormatApi={1}") public static Iterable constructorFeeder() { - NestedDataModule.registerHandlersAndSerde(); OBJECT_MAPPER.registerModules(new SamplerModule().getJacksonModules()); - OBJECT_MAPPER.registerModules(NestedDataModule.getJacksonModulesList()); return ImmutableList.of( new Object[]{ParserType.STR_JSON, false}, new Object[]{ParserType.STR_JSON, true}, @@ -182,14 +173,12 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest @Test public void testNoDataSchema() { - final InputSource inputSource = createInputSource(getTestRows(), null); + final InputSource inputSource = createInputSource(getTestRows()); final SamplerResponse response = inputSourceSampler.sample(inputSource, createInputFormat(), null, null); Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(0, response.getNumRowsIndexed()); Assert.assertEquals(6, response.getData().size()); - Assert.assertEquals(ImmutableList.of(), response.getLogicalDimensions()); - Assert.assertEquals(RowSignature.builder().addTimeColumn().build(), response.getLogicalSegmentSchema()); List data = response.getData(); @@ -252,7 +241,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest @Test public void testNoDataSchemaNumRows() { - final InputSource inputSource = createInputSource(getTestRows(), null); + final InputSource inputSource = createInputSource(getTestRows()); final SamplerResponse response = inputSourceSampler.sample( inputSource, createInputFormat(), @@ -263,8 +252,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest Assert.assertEquals(3, response.getNumRowsRead()); Assert.assertEquals(0, response.getNumRowsIndexed()); Assert.assertEquals(3, response.getData().size()); - Assert.assertEquals(ImmutableList.of(), response.getLogicalDimensions()); - Assert.assertEquals(RowSignature.builder().addTimeColumn().build(), response.getLogicalSegmentSchema()); List data = response.getData(); @@ -303,7 +290,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest final TimestampSpec timestampSpec = new TimestampSpec(null, null, DateTimes.of("1970")); final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); final DataSchema dataSchema = createDataSchema(timestampSpec, dimensionsSpec, null, null, null); - final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputSource inputSource = createInputSource(getTestRows()); final InputFormat inputFormat = createInputFormat(); SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); @@ -311,40 +298,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(6, response.getNumRowsIndexed()); Assert.assertEquals(6, response.getData().size()); - Assert.assertEquals( - parserType == ParserType.STR_JSON - ? ImmutableList.of( - new StringDimensionSchema("t"), - new StringDimensionSchema("dim1"), - new StringDimensionSchema("met1"), - new StringDimensionSchema("dim2") - ) - : ImmutableList.of( - new StringDimensionSchema("t"), - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim2"), - new StringDimensionSchema("met1") - ), - response.getLogicalDimensions() - ); - Assert.assertEquals( - parserType == ParserType.STR_JSON - ? RowSignature.builder() - .addTimeColumn() - .add("t", ColumnType.STRING) - .add("dim1", ColumnType.STRING) - .add("met1", ColumnType.STRING) - .add("dim2", ColumnType.STRING) - .build() - : RowSignature.builder() - .addTimeColumn() - .add("t", ColumnType.STRING) - .add("dim1", ColumnType.STRING) - .add("dim2", ColumnType.STRING) - .add("met1", ColumnType.STRING) - .build(), - response.getLogicalSegmentSchema() - ); List data = response.getData(); @@ -446,7 +399,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); final DataSchema dataSchema = createDataSchema(timestampSpec, dimensionsSpec, null, null, null); - final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputSource inputSource = createInputSource(getTestRows()); final InputFormat inputFormat = createInputFormat(); SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); @@ -454,36 +407,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); Assert.assertEquals(6, response.getData().size()); - Assert.assertEquals( - parserType == ParserType.STR_JSON - ? ImmutableList.of( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("met1"), - new StringDimensionSchema("dim2") - ) - : ImmutableList.of( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim2"), - new StringDimensionSchema("met1") - ), - response.getLogicalDimensions() - ); - Assert.assertEquals( - parserType == ParserType.STR_JSON - ? RowSignature.builder() - .addTimeColumn() - .add("dim1", ColumnType.STRING) - .add("met1", ColumnType.STRING) - .add("dim2", ColumnType.STRING) - .build() - : RowSignature.builder() - .addTimeColumn() - .add("dim1", ColumnType.STRING) - .add("dim2", ColumnType.STRING) - .add("met1", ColumnType.STRING) - .build(), - response.getLogicalSegmentSchema() - ); List data = response.getData(); @@ -568,137 +491,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest ); } - @Test - public void testWithTimestampSpecNestedDiscovery() throws IOException - { - - final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); - final DimensionsSpec dimensionsSpec = DimensionsSpec.builder() - .useSchemaDiscovery(true) - .build(); - final DataSchema dataSchema = createDataSchema(timestampSpec, dimensionsSpec, null, null, null); - final InputSource inputSource = createInputSource(getTestRows(), dataSchema); - final InputFormat inputFormat = createInputFormat(); - - SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); - - Assert.assertEquals(6, response.getNumRowsRead()); - Assert.assertEquals(5, response.getNumRowsIndexed()); - Assert.assertEquals(6, response.getData().size()); - Assert.assertEquals( - parserType == ParserType.STR_JSON - ? ImmutableList.of( - new StringDimensionSchema("dim1"), - new LongDimensionSchema("met1"), - new StringDimensionSchema("dim2") - ) - : ImmutableList.of( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim2"), - new StringDimensionSchema("met1") - ), - response.getLogicalDimensions() - ); - Assert.assertEquals( - parserType == ParserType.STR_JSON - ? RowSignature.builder() - .addTimeColumn() - .add("dim1", ColumnType.STRING) - .add("met1", ColumnType.LONG) - .add("dim2", ColumnType.STRING) - .build() - : RowSignature.builder() - .addTimeColumn() - .add("dim1", ColumnType.STRING) - .add("dim2", ColumnType.STRING) - .add("met1", ColumnType.STRING) - .build(), - response.getLogicalSegmentSchema() - ); - - List data = response.getData(); - - assertEqualsSamplerResponseRow( - new SamplerResponseRow( - getRawColumns().get(0), - new SamplerTestUtils.MapAllowingNullValuesBuilder() - .put("__time", 1555934400000L) - .put("dim2", StructuredData.wrap(null)) - .put("dim1", StructuredData.wrap("foo")) - .put("met1", StructuredData.wrap(parserType == ParserType.STR_JSON ? 1L : "1")) - .build(), - null, - null - ), - data.get(0) - ); - assertEqualsSamplerResponseRow( - new SamplerResponseRow( - getRawColumns().get(1), - new SamplerTestUtils.MapAllowingNullValuesBuilder() - .put("__time", 1555934400000L) - .put("dim2", StructuredData.wrap(null)) - .put("dim1", StructuredData.wrap("foo")) - .put("met1", StructuredData.wrap(parserType == ParserType.STR_JSON ? 2L : "2")) - .build(), - null, - null - ), - data.get(1) - ); - assertEqualsSamplerResponseRow( - new SamplerResponseRow( - getRawColumns().get(2), - new SamplerTestUtils.MapAllowingNullValuesBuilder() - .put("__time", 1555934460000L) - .put("dim2", StructuredData.wrap(null)) - .put("dim1", StructuredData.wrap("foo")) - .put("met1", StructuredData.wrap(parserType == ParserType.STR_JSON ? 3L : "3")) - .build(), - null, - null - ), - data.get(2) - ); - assertEqualsSamplerResponseRow( - new SamplerResponseRow( - getRawColumns().get(3), - new SamplerTestUtils.MapAllowingNullValuesBuilder() - .put("__time", 1555934400000L) - .put("dim2", StructuredData.wrap(null)) - .put("dim1", StructuredData.wrap("foo2")) - .put("met1", StructuredData.wrap(parserType == ParserType.STR_JSON ? 4L : "4")) - .build(), - null, - null - ), - data.get(3) - ); - assertEqualsSamplerResponseRow( - new SamplerResponseRow( - getRawColumns().get(4), - new SamplerTestUtils.MapAllowingNullValuesBuilder() - .put("__time", 1555934400000L) - .put("dim2", StructuredData.wrap("bar")) - .put("dim1", StructuredData.wrap("foo")) - .put("met1", StructuredData.wrap(parserType == ParserType.STR_JSON ? 5L : "5")) - .build(), - null, - null - ), - data.get(4) - ); - assertEqualsSamplerResponseRow( - new SamplerResponseRow( - getRawColumns().get(5), - null, - true, - getUnparseableTimestampString() - ), - data.get(5) - ); - } - @Test public void testWithDimensionSpec() throws IOException { @@ -707,7 +499,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest ImmutableList.of(StringDimensionSchema.create("dim1"), StringDimensionSchema.create("met1")) ); final DataSchema dataSchema = createDataSchema(timestampSpec, dimensionsSpec, null, null, null); - final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputSource inputSource = createInputSource(getTestRows()); final InputFormat inputFormat = createInputFormat(); SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); @@ -715,21 +507,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); Assert.assertEquals(6, response.getData().size()); - Assert.assertEquals( - ImmutableList.of( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("met1") - ), - response.getLogicalDimensions() - ); - Assert.assertEquals( - RowSignature.builder() - .addTimeColumn() - .add("dim1", ColumnType.STRING) - .add("met1", ColumnType.STRING) - .build(), - response.getLogicalSegmentSchema() - ); List data = response.getData(); @@ -828,7 +605,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest granularitySpec, null ); - final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputSource inputSource = createInputSource(getTestRows()); final InputFormat inputFormat = createInputFormat(); SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); @@ -836,22 +613,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); Assert.assertEquals(6, response.getData().size()); - Assert.assertEquals( - ImmutableList.of( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim2") - ), - response.getLogicalDimensions() - ); - Assert.assertEquals( - RowSignature.builder() - .addTimeColumn() - .add("dim1", ColumnType.STRING) - .add("dim2", ColumnType.STRING) - .add("met1", ColumnType.LONG) - .build(), - response.getLogicalSegmentSchema() - ); List data = response.getData(); @@ -955,7 +716,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest granularitySpec, null ); - final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputSource inputSource = createInputSource(getTestRows()); final InputFormat inputFormat = createInputFormat(); SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); @@ -963,22 +724,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); Assert.assertEquals(4, response.getData().size()); - Assert.assertEquals( - ImmutableList.of( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim2") - ), - response.getLogicalDimensions() - ); - Assert.assertEquals( - RowSignature.builder() - .addTimeColumn() - .add("dim1", ColumnType.STRING) - .add("dim2", ColumnType.STRING) - .add("met1", ColumnType.LONG) - .build(), - response.getLogicalSegmentSchema() - ); List data = response.getData(); @@ -1054,7 +799,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest granularitySpec, null ); - final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputSource inputSource = createInputSource(getTestRows()); final InputFormat inputFormat = createInputFormat(); SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); @@ -1062,20 +807,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); Assert.assertEquals(3, response.getData().size()); - Assert.assertEquals( - ImmutableList.of( - new StringDimensionSchema("dim1") - ), - response.getLogicalDimensions() - ); - Assert.assertEquals( - RowSignature.builder() - .addTimeColumn() - .add("dim1", ColumnType.STRING) - .add("met1", ColumnType.LONG) - .build(), - response.getLogicalSegmentSchema() - ); List data = response.getData(); @@ -1139,7 +870,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest granularitySpec, transformSpec ); - final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputSource inputSource = createInputSource(getTestRows()); final InputFormat inputFormat = createInputFormat(); SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); @@ -1147,22 +878,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); Assert.assertEquals(4, response.getData().size()); - Assert.assertEquals( - ImmutableList.of( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim2") - ), - response.getLogicalDimensions() - ); - Assert.assertEquals( - RowSignature.builder() - .addTimeColumn() - .add("dim1", ColumnType.STRING) - .add("dim2", ColumnType.STRING) - .add("met1", ColumnType.LONG) - .build(), - response.getLogicalSegmentSchema() - ); List data = response.getData(); @@ -1244,7 +959,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest granularitySpec, transformSpec ); - final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputSource inputSource = createInputSource(getTestRows()); final InputFormat inputFormat = createInputFormat(); SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); @@ -1252,20 +967,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest Assert.assertEquals(6, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); Assert.assertEquals(3, response.getData().size()); - Assert.assertEquals( - ImmutableList.of( - new StringDimensionSchema("dim1PlusBar") - ), - response.getLogicalDimensions() - ); - Assert.assertEquals( - RowSignature.builder() - .addTimeColumn() - .add("dim1PlusBar", ColumnType.STRING) - .add("met1", ColumnType.LONG) - .build(), - response.getLogicalSegmentSchema() - ); List data = response.getData(); @@ -1326,7 +1027,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest granularitySpec, transformSpec ); - final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputSource inputSource = createInputSource(getTestRows()); final InputFormat inputFormat = createInputFormat(); SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); @@ -1334,22 +1035,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest Assert.assertEquals(5, response.getNumRowsRead()); Assert.assertEquals(4, response.getNumRowsIndexed()); Assert.assertEquals(3, response.getData().size()); - Assert.assertEquals( - ImmutableList.of( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim2") - ), - response.getLogicalDimensions() - ); - Assert.assertEquals( - RowSignature.builder() - .addTimeColumn() - .add("dim1", ColumnType.STRING) - .add("dim2", ColumnType.STRING) - .add("met1", ColumnType.LONG) - .build(), - response.getLogicalSegmentSchema() - ); List data = response.getData(); @@ -1423,14 +1108,13 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest // Map rawColumns4ParseExceptionRow = ImmutableMap.of("t", "2019-04-22T12:00", "dim1", "foo2", - "met1", "invalidNumber" - ); + "met1", "invalidNumber"); final List inputTestRows = Lists.newArrayList(getTestRows()); inputTestRows.add(ParserType.STR_CSV.equals(parserType) ? "2019-04-22T12:00,foo2,,invalidNumber" : OBJECT_MAPPER.writeValueAsString(rawColumns4ParseExceptionRow)); - final InputSource inputSource = createInputSource(inputTestRows, dataSchema); + final InputSource inputSource = createInputSource(inputTestRows); final InputFormat inputFormat = createInputFormat(); SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); @@ -1438,20 +1122,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest Assert.assertEquals(7, response.getNumRowsRead()); Assert.assertEquals(5, response.getNumRowsIndexed()); Assert.assertEquals(4, response.getData().size()); - Assert.assertEquals( - ImmutableList.of( - new StringDimensionSchema("dim1PlusBar") - ), - response.getLogicalDimensions() - ); - Assert.assertEquals( - RowSignature.builder() - .addTimeColumn() - .add("dim1PlusBar", ColumnType.STRING) - .add("met1", ColumnType.LONG) - .build(), - response.getLogicalSegmentSchema() - ); List data = response.getData(); @@ -1495,8 +1165,8 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest // the last row has parse exception when indexing, check if rawColumns and exception message match the expected // String indexParseExceptioMessage = ParserType.STR_CSV.equals(parserType) - ? "Found unparseable columns in row: [SamplerInputRow{row=TransformedInputRow{row=MapBasedInputRow{timestamp=2019-04-22T12:00:00.000Z, event={t=2019-04-22T12:00, dim1=foo2, dim2=null, met1=invalidNumber}, dimensions=[dim1PlusBar]}}}], exceptions: [Unable to parse value[invalidNumber] for field[met1]]" - : "Found unparseable columns in row: [SamplerInputRow{row=TransformedInputRow{row=MapBasedInputRow{timestamp=2019-04-22T12:00:00.000Z, event={t=2019-04-22T12:00, dim1=foo2, met1=invalidNumber}, dimensions=[dim1PlusBar]}}}], exceptions: [Unable to parse value[invalidNumber] for field[met1]]"; + ? "Found unparseable columns in row: [SamplerInputRow{row=TransformedInputRow{row=MapBasedInputRow{timestamp=2019-04-22T12:00:00.000Z, event={t=2019-04-22T12:00, dim1=foo2, dim2=null, met1=invalidNumber}, dimensions=[dim1PlusBar]}}}], exceptions: [Unable to parse value[invalidNumber] for field[met1]]" + : "Found unparseable columns in row: [SamplerInputRow{row=TransformedInputRow{row=MapBasedInputRow{timestamp=2019-04-22T12:00:00.000Z, event={t=2019-04-22T12:00, dim1=foo2, met1=invalidNumber}, dimensions=[dim1PlusBar]}}}], exceptions: [Unable to parse value[invalidNumber] for field[met1]]"; assertEqualsSamplerResponseRow( new SamplerResponseRow( rawColumns4ParseExceptionRow, @@ -1509,13 +1179,15 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest } /** + * * This case tests sampling for multiple json lines in one text block * Currently only RecordSupplierInputSource supports this kind of input, see https://github.com/apache/druid/pull/10383 for more information - *

+ * * This test combines illegal json block and legal json block together to verify: * 1. all lines in the illegal json block should not be parsed * 2. the illegal json block should not affect the processing of the 2nd record * 3. all lines in legal json block should be parsed successfully + * */ @Test public void testMultipleJsonStringInOneBlock() throws IOException @@ -1571,20 +1243,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest Assert.assertEquals(illegalRows + legalRows, response.getNumRowsRead()); Assert.assertEquals(legalRows, response.getNumRowsIndexed()); Assert.assertEquals(illegalRows + 2, response.getData().size()); - Assert.assertEquals( - ImmutableList.of( - new StringDimensionSchema("dim1PlusBar") - ), - response.getLogicalDimensions() - ); - Assert.assertEquals( - RowSignature.builder() - .addTimeColumn() - .add("dim1PlusBar", ColumnType.STRING) - .add("met1", ColumnType.LONG) - .build(), - response.getLogicalSegmentSchema() - ); List data = response.getData(); List> rawColumnList = this.getRawColumns(); @@ -1594,11 +1252,8 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest // first n rows are related to the first json block which fails to parse // String parseExceptionMessage; - if (useInputFormatApi) { - parseExceptionMessage = "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, met1=6}"; - } else { - parseExceptionMessage = "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, met1=6}"; - } + parseExceptionMessage = "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, met1=6}"; + for (; index < illegalRows; index++) { assertEqualsSamplerResponseRow( new SamplerResponseRow( @@ -1691,7 +1346,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest granularitySpec, null ); - final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputSource inputSource = createInputSource(getTestRows()); final InputFormat inputFormat = createInputFormat(); SamplerResponse response = inputSourceSampler.sample( @@ -1704,33 +1359,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest Assert.assertEquals(4, response.getNumRowsRead()); Assert.assertEquals(4, response.getNumRowsIndexed()); Assert.assertEquals(2, response.getData().size()); - Assert.assertEquals( - parserType == ParserType.STR_JSON - ? ImmutableList.of( - new StringDimensionSchema("dim1") - ) - : ImmutableList.of( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim2") - ), - - response.getLogicalDimensions() - ); - Assert.assertEquals( - parserType == ParserType.STR_JSON - ? RowSignature.builder() - .addTimeColumn() - .add("dim1", ColumnType.STRING) - .add("met1", ColumnType.LONG) - .build() - : RowSignature.builder() - .addTimeColumn() - .add("dim1", ColumnType.STRING) - .add("dim2", ColumnType.STRING) - .add("met1", ColumnType.LONG) - .build(), - response.getLogicalSegmentSchema() - ); } @@ -1753,7 +1381,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest granularitySpec, null ); - final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputSource inputSource = createInputSource(getTestRows()); final InputFormat inputFormat = createInputFormat(); SamplerResponse response = inputSourceSampler.sample( @@ -1766,32 +1394,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest Assert.assertEquals(4, response.getNumRowsRead()); Assert.assertEquals(4, response.getNumRowsIndexed()); Assert.assertEquals(2, response.getData().size()); - Assert.assertEquals( - parserType == ParserType.STR_JSON - ? ImmutableList.of( - new StringDimensionSchema("dim1") - ) - : ImmutableList.of( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim2") - ), - response.getLogicalDimensions() - ); - Assert.assertEquals( - parserType == ParserType.STR_JSON - ? RowSignature.builder() - .addTimeColumn() - .add("dim1", ColumnType.STRING) - .add("met1", ColumnType.LONG) - .build() - : RowSignature.builder() - .addTimeColumn() - .add("dim1", ColumnType.STRING) - .add("dim2", ColumnType.STRING) - .add("met1", ColumnType.LONG) - .build(), - response.getLogicalSegmentSchema() - ); } @Test @@ -1813,7 +1415,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest granularitySpec, null ); - final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputSource inputSource = createInputSource(getTestRows()); final InputFormat inputFormat = createInputFormat(); SamplerResponse response = inputSourceSampler.sample( @@ -1826,22 +1428,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest Assert.assertEquals(4, response.getNumRowsRead()); Assert.assertEquals(4, response.getNumRowsIndexed()); Assert.assertEquals(2, response.getData().size()); - Assert.assertEquals( - ImmutableList.of( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim2") - ), - response.getLogicalDimensions() - ); - Assert.assertEquals( - RowSignature.builder() - .addTimeColumn() - .add("dim1", ColumnType.STRING) - .add("dim2", ColumnType.STRING) - .add("met1", ColumnType.LONG) - .build(), - response.getLogicalSegmentSchema() - ); } private List getTestRows() @@ -1940,42 +1526,22 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest return OBJECT_MAPPER.readValue(OBJECT_MAPPER.writeValueAsBytes(parser), Map.class); } - private InputSource createInputSource(List rows, DataSchema dataSchema) + private InputSource createInputSource(List rows) { final String data = String.join("\n", rows); - if (useInputFormatApi) { - return new InlineInputSource(data); - } else { - return new FirehoseFactoryToInputSourceAdaptor( - new InlineFirehoseFactory(data), - createInputRowParser( - dataSchema == null ? new TimestampSpec(null, null, null) : dataSchema.getTimestampSpec(), - dataSchema == null ? new DimensionsSpec(null) : dataSchema.getDimensionsSpec() - ) - ); - } + return new InlineInputSource(data); } private String getUnparseableTimestampString() { - if (useInputFormatApi) { - return ParserType.STR_CSV.equals(parserType) - ? "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, dim2=null, met1=6} (Line: 6)" - : "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, met1=6} (Line: 6)"; - } else { - return ParserType.STR_CSV.equals(parserType) - ? "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, dim2=null, met1=6}" - : "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, met1=6}"; - } + return ParserType.STR_CSV.equals(parserType) + ? "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, dim2=null, met1=6} (Line: 6)" + : "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, met1=6} (Line: 6)"; } private String unparseableTimestampErrorString(Map rawColumns, int line) { - if (useInputFormatApi) { - return StringUtils.format("Timestamp[null] is unparseable! Event: %s (Line: %d)", rawColumns, line); - } else { - return StringUtils.format("Timestamp[null] is unparseable! Event: %s", rawColumns); - } + return StringUtils.format("Timestamp[null] is unparseable! Event: %s (Line: %d)", rawColumns, line); } @Nullable diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java new file mode 100644 index 00000000000..8bc91f7c1fa --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java @@ -0,0 +1,354 @@ +/* + * 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.druid.indexing.seekablestream; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.client.indexing.SamplerResponse; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.FloatDimensionSchema; +import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.LongDimensionSchema; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; +import org.apache.druid.indexing.overlord.sampler.SamplerConfig; +import org.apache.druid.indexing.overlord.sampler.SamplerTestUtils; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.indexing.seekablestream.supervisor.IdleConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; +import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.easymock.EasyMock; +import org.easymock.EasyMockSupport; +import org.joda.time.DateTime; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +public class SeekableStreamSamplerSpecTest extends EasyMockSupport +{ + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); + private static final String STREAM = "sampling"; + private static final String SHARD_ID = "1"; + + private final SeekableStreamSupervisorSpec supervisorSpec = mock(SeekableStreamSupervisorSpec.class); + + static { + NullHandling.initializeForTests(); + } + + private final RecordSupplier recordSupplier = mock(RecordSupplier.class); + + private static List> generateRecords(String stream) + { + return ImmutableList.of( + new OrderedPartitionableRecord<>(stream, "1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "1", jb("2009", "b", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "2", jb("2010", "c", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>( + stream, + "1", + "5", + jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0") + ), + new OrderedPartitionableRecord<>( + stream, + "1", + "6", + Collections.singletonList(new ByteEntity(StringUtils.toUtf8("unparseable"))) + ), + new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(new ByteEntity(StringUtils.toUtf8("{}")))) + ); + } + + @Test(timeout = 10_000L) + public void testSampleWithInputRowParser() throws Exception + { + final DataSchema dataSchema = new DataSchema( + "test_ds", + OBJECT_MAPPER.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ) + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of(), + false + ) + ), + Map.class + ), + new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + }, + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), + null, + OBJECT_MAPPER + ); + + final SeekableStreamSupervisorIOConfig supervisorIOConfig = new TestableSeekableStreamSupervisorIOConfig( + STREAM, + null, + null, + null, + null, + null, + null, + true, + null, + null, + null, + null, + null, + null + ); + + EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).once(); + EasyMock.expect(supervisorSpec.getDataSchema()).andReturn(dataSchema).once(); + EasyMock.expect(supervisorSpec.getIoConfig()).andReturn(supervisorIOConfig).once(); + EasyMock.expect(supervisorSpec.getTuningConfig()).andReturn(null).once(); + + recordSupplier.assign(ImmutableSet.of(StreamPartition.of(STREAM, SHARD_ID))); + EasyMock.expectLastCall().once(); + + recordSupplier.seekToEarliest(ImmutableSet.of(StreamPartition.of(STREAM, SHARD_ID))); + EasyMock.expectLastCall().once(); + + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(STREAM)).once(); + + recordSupplier.close(); + EasyMock.expectLastCall().once(); + + replayAll(); + + SeekableStreamSamplerSpec samplerSpec = new TestableSeekableStreamSamplerSpec( + supervisorSpec, + new SamplerConfig(5, null, null, null), + new InputSourceSampler(new DefaultObjectMapper()) + ); + + SamplerResponse response = samplerSpec.sample(); + + verifyAll(); + + Assert.assertEquals(5, response.getNumRowsRead()); + Assert.assertEquals(3, response.getNumRowsIndexed()); + Assert.assertEquals(5, response.getData().size()); + + Iterator it = response.getData().iterator(); + + Assert.assertEquals(new SamplerResponse.SamplerResponseRow( + ImmutableMap.builder() + .put("timestamp", "2008") + .put("dim1", "a") + .put("dim2", "y") + .put("dimLong", "10") + .put("dimFloat", "20.0") + .put("met1", "1.0") + .build(), + new SamplerTestUtils.MapAllowingNullValuesBuilder() + .put("__time", 1199145600000L) + .put("dim1", "a") + .put("dim1t", null) + .put("dim2", "y") + .put("dimLong", 10L) + .put("dimFloat", 20.0F) + .put("rows", 1L) + .put("met1sum", 1.0) + .build(), + null, + null + ), it.next()); + Assert.assertEquals(new SamplerResponse.SamplerResponseRow( + ImmutableMap.builder() + .put("timestamp", "2009") + .put("dim1", "b") + .put("dim2", "y") + .put("dimLong", "10") + .put("dimFloat", "20.0") + .put("met1", "1.0") + .build(), + new SamplerTestUtils.MapAllowingNullValuesBuilder() + .put("__time", 1230768000000L) + .put("dim1", "b") + .put("dim1t", null) + .put("dim2", "y") + .put("dimLong", 10L) + .put("dimFloat", 20.0F) + .put("rows", 1L) + .put("met1sum", 1.0) + .build(), + null, + null + ), it.next()); + Assert.assertEquals(new SamplerResponse.SamplerResponseRow( + ImmutableMap.builder() + .put("timestamp", "2010") + .put("dim1", "c") + .put("dim2", "y") + .put("dimLong", "10") + .put("dimFloat", "20.0") + .put("met1", "1.0") + .build(), + new SamplerTestUtils.MapAllowingNullValuesBuilder() + .put("__time", 1262304000000L) + .put("dim1", "c") + .put("dim1t", null) + .put("dim2", "y") + .put("dimLong", 10L) + .put("dimFloat", 20.0F) + .put("rows", 1L) + .put("met1sum", 1.0) + .build(), + null, + null + ), it.next()); + Assert.assertEquals(new SamplerResponse.SamplerResponseRow( + ImmutableMap.builder() + .put("timestamp", "246140482-04-24T15:36:27.903Z") + .put("dim1", "x") + .put("dim2", "z") + .put("dimLong", "10") + .put("dimFloat", "20.0") + .put("met1", "1.0") + .build(), + null, + true, + "Encountered row with timestamp[246140482-04-24T15:36:27.903Z] that cannot be represented as a long: [{timestamp=246140482-04-24T15:36:27.903Z, dim1=x, dim2=z, dimLong=10, dimFloat=20.0, met1=1.0}]" + ), it.next()); + Assert.assertEquals(new SamplerResponse.SamplerResponseRow( + null, + null, + true, + "Unable to parse row [unparseable]" + ), it.next()); + + Assert.assertFalse(it.hasNext()); + } + + private static List jb(String ts, String dim1, String dim2, String dimLong, String dimFloat, String met1) + { + try { + return Collections.singletonList(new ByteEntity(new ObjectMapper().writeValueAsBytes( + ImmutableMap.builder() + .put("timestamp", ts) + .put("dim1", dim1) + .put("dim2", dim2) + .put("dimLong", dimLong) + .put("dimFloat", dimFloat) + .put("met1", met1) + .build() + ))); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + private class TestableSeekableStreamSamplerSpec extends SeekableStreamSamplerSpec + { + private TestableSeekableStreamSamplerSpec( + SeekableStreamSupervisorSpec ingestionSpec, + SamplerConfig samplerConfig, + InputSourceSampler inputSourceSampler + ) + { + super(ingestionSpec, samplerConfig, inputSourceSampler); + } + + @Override + protected RecordSupplier createRecordSupplier() + { + return recordSupplier; + } + } + + private static class TestableSeekableStreamSupervisorIOConfig extends SeekableStreamSupervisorIOConfig + { + private TestableSeekableStreamSupervisorIOConfig( + String stream, + @Nullable InputFormat inputFormat, + Integer replicas, + Integer taskCount, + Period taskDuration, + Period startDelay, + Period period, + Boolean useEarliestSequenceNumber, + Period completionTimeout, + Period lateMessageRejectionPeriod, + Period earlyMessageRejectionPeriod, + @Nullable AutoScalerConfig autoScalerConfig, + DateTime lateMessageRejectionStartDateTime, + @Nullable IdleConfig idleConfig + ) + { + super( + stream, + inputFormat, + replicas, + taskCount, + taskDuration, + startDelay, + period, + useEarliestSequenceNumber, + completionTimeout, + lateMessageRejectionPeriod, + earlyMessageRejectionPeriod, + autoScalerConfig, + lateMessageRejectionStartDateTime, + idleConfig + ); + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java index 3eed893b372..c61bb40bfd7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java @@ -31,12 +31,10 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.realtime.FireDepartment; -import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; +import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; -import java.io.File; - public class TaskAnnouncementTest { private final ObjectMapper jsonMapper; @@ -50,15 +48,13 @@ public class TaskAnnouncementTest @Test public void testBackwardsCompatibleSerde() throws Exception { + final RealtimeIOConfig realtimeIOConfig = EasyMock.createNiceMock(RealtimeIOConfig.class); final Task task = new RealtimeIndexTask( "theid", new TaskResource("rofl", 2), new FireDepartment( new DataSchema("foo", null, new AggregatorFactory[0], null, null, new DefaultObjectMapper()), - new RealtimeIOConfig( - new LocalFirehoseFactory(new File("lol"), "rofl", null), - (schema, config, metrics) -> null - ), + realtimeIOConfig, null ), null diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java index 1fc1d3cfe10..339f6af9485 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -34,7 +34,7 @@ import org.apache.druid.indexing.common.IndexingServiceCondition; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TaskStorageDirTracker; import org.apache.druid.indexing.common.TaskToolboxFactory; -import org.apache.druid.indexing.common.TestRealtimeTask; +import org.apache.druid.indexing.common.TestIndexTask; import org.apache.druid.indexing.common.TestTasks; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.actions.TaskActionClient; @@ -146,7 +146,7 @@ public class WorkerTaskMonitorTest // Start a task monitor workerTaskMonitor = createTaskMonitor(); TestTasks.registerSubtypes(jsonMapper); - jsonMapper.registerSubtypes(new NamedType(TestRealtimeTask.class, "test_realtime")); + jsonMapper.registerSubtypes(new NamedType(TestIndexTask.class, "test_index")); workerTaskMonitor.start(); task = TestTasks.immediateSuccess("test"); diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningFirehoseFactoryIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningFirehoseFactoryIndexTest.java deleted file mode 100644 index 8d394a0297e..00000000000 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningFirehoseFactoryIndexTest.java +++ /dev/null @@ -1,101 +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.druid.testsEx.indexer; - -import com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.testsEx.categories.BatchIndex; -import org.apache.druid.testsEx.config.DruidTestRunner; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; - -import java.io.Closeable; -import java.util.function.Function; - -@RunWith(DruidTestRunner.class) -@Category(BatchIndex.class) -public class ITCombiningFirehoseFactoryIndexTest extends AbstractITBatchIndexTest -{ - private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json"; - private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; - private static final String INDEX_DATASOURCE = "wikipedia_index_test"; - - private static final String COMBINING_INDEX_TASK = "/indexer/wikipedia_combining_firehose_index_task.json"; - private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_firehose_index_queries.json"; - private static final String COMBINING_INDEX_DATASOURCE = "wikipedia_comb_index_test"; - - @Test - public void testIndexData() throws Exception - { - try ( - final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); - final Closeable ignored2 = unloader(COMBINING_INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); - ) { - final Function combiningFirehoseSpecTransform = spec -> { - try { - return StringUtils.replace( - spec, - "%%COMBINING_DATASOURCE%%", - INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix() - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - }; - final Function transform = spec -> { - try { - return StringUtils.replace( - spec, - "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", - jsonMapper.writeValueAsString("0") - ); - } - catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - }; - - doIndexTest( - INDEX_DATASOURCE, - INDEX_TASK, - transform, - INDEX_QUERIES_RESOURCE, - false, - true, - true, - new Pair<>(false, false) - ); - doIndexTest( - COMBINING_INDEX_DATASOURCE, - COMBINING_INDEX_TASK, - combiningFirehoseSpecTransform, - COMBINING_QUERIES_RESOURCE, - false, - true, - true, - new Pair<>(false, false) - ); - } - } - -} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningInputSourceParallelIndexTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningInputSourceParallelIndexTest.java index 5ec9ea0dbc4..75b5a0e2815 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningInputSourceParallelIndexTest.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITCombiningInputSourceParallelIndexTest.java @@ -42,7 +42,7 @@ public class ITCombiningInputSourceParallelIndexTest extends AbstractITBatchInde private static final String INDEX_DATASOURCE = "wikipedia_index_test"; private static final String COMBINING_INDEX_TASK = "/indexer/wikipedia_combining_input_source_index_parallel_task.json"; - private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_firehose_index_queries.json"; + private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_input_source_index_queries.json"; private static final String COMBINING_INDEX_DATASOURCE = "wikipedia_comb_index_test"; @Test diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITTransformTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITTransformTest.java index 61cb33fd2c6..f8769461f6a 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITTransformTest.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITTransformTest.java @@ -22,7 +22,6 @@ package org.apache.druid.testsEx.indexer; import org.apache.druid.java.util.common.Pair; import org.apache.druid.testsEx.categories.BatchIndex; import org.apache.druid.testsEx.config.DruidTestRunner; -import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -34,12 +33,10 @@ import java.io.IOException; @Category(BatchIndex.class) public class ITTransformTest extends AbstractITBatchIndexTest { - private static final String INDEX_TASK_WITH_FIREHOSE = "/indexer/wikipedia_index_task_with_transform.json"; private static final String INDEX_TASK_WITH_INPUT_SOURCE = "/indexer/wikipedia_index_task_with_inputsource_transform.json"; private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries_with_transform.json"; private static final String INDEX_DATASOURCE = "wikipedia_index_test"; - private static final String REINDEX_TASK = "/indexer/wikipedia_reindex_task_with_transforms.json"; private static final String REINDEX_TASK_WITH_DRUID_INPUT_SOURCE = "/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json"; private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_reindex_queries_with_transforms.json"; private static final String REINDEX_DATASOURCE = "wikipedia_reindex_test"; @@ -71,55 +68,4 @@ public class ITTransformTest extends AbstractITBatchIndexTest ); } } - - @Test - @Ignore - public void testIndexAndReIndexUsingIngestSegmentWithTransforms() throws IOException - { - // TODO: re-instate this test when https://github.com/apache/druid/issues/9591 is fixed - // Move the re-index step into testIndexAndReIndexWithTransformSpec for faster tests! - final String reindexDatasource = REINDEX_DATASOURCE + "-testIndexData"; - try ( - final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); - final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix()) - ) { - doIndexTest( - INDEX_DATASOURCE, - INDEX_TASK_WITH_INPUT_SOURCE, - INDEX_QUERIES_RESOURCE, - false, - true, - true, - new Pair<>(false, false) - ); - doReindexTest( - INDEX_DATASOURCE, - reindexDatasource, - REINDEX_TASK, - REINDEX_QUERIES_RESOURCE, - new Pair<>(false, false) - ); - } - } - - @Test - @Ignore - public void testIndexWithFirehoseAndTransforms() throws IOException - { - // TODO: re-instate this test when https://github.com/apache/druid/issues/9589 is fixed - final String indexDatasource = INDEX_DATASOURCE + "-firehose"; - try ( - final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix()); - ) { - doIndexTest( - indexDatasource, - INDEX_TASK_WITH_FIREHOSE, - INDEX_QUERIES_RESOURCE, - false, - true, - true, - new Pair<>(false, false) - ); - } - } } diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json deleted file mode 100644 index 1e7deffc6b5..00000000000 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json +++ /dev/null @@ -1,95 +0,0 @@ -{ - "type": "index", - "spec": { - "dataSchema": { - "dataSource": "%%DATASOURCE%%", - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - }, - { - "name": "thetaSketch", - "type": "thetaSketch", - "fieldName": "user" - }, - { - "name": "quantilesDoublesSketch", - "type": "quantilesDoublesSketch", - "fieldName": "delta" - }, - { - "name": "HLLSketchBuild", - "type": "HLLSketchBuild", - "fieldName": "user" - } - ], - "granularitySpec": { - "segmentGranularity": "DAY", - "queryGranularity": "second", - "intervals" : [ "2013-08-31/2013-09-02" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } - } - }, - "ioConfig": { - "type": "index", - "firehose": { - "type": "combining", - "delegates": [ - { - "type": "local", - "baseDir": "/resources/indexer", - "filter": "wikipedia_combining_index_data.json" - }, - { - "type": "ingestSegment", - "dataSource": "%%COMBINING_DATASOURCE%%", - "interval": "2013-08-31/2013-09-02" - } - ] - } - }, - "tuningConfig": { - "type": "index", - "maxRowsPerSegment": 3 - } - } -} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_queries.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_input_source_index_queries.json similarity index 100% rename from integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_firehose_index_queries.json rename to integration-tests-ex/cases/src/test/resources/indexer/wikipedia_combining_input_source_index_queries.json diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json index 1fa8b4eba32..35b7e61daae 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json @@ -3,6 +3,26 @@ "spec": { "dataSchema": { "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, "metricsSpec": [ { "type": "count", @@ -43,39 +63,22 @@ "segmentGranularity": "DAY", "queryGranularity": "DAY", "intervals" : [ "2013-08-31/2013-09-02" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } } }, "ioConfig": { "type": "index", - "firehose": { + "inputSource": { "type": "local", - "baseDir": "/resources/data/batch_index/json", - "filter": "wikipedia_index_data*" - } + "files": [ + "/resources/data/batch_index/json/wikipedia_index_data1.json", + "/resources/data/batch_index/json/wikipedia_index_data2.json", + "/resources/data/batch_index/json/wikipedia_index_data3.json" + ] + }, + "inputFormat": { + "type": "json" + }, + "appendToExisting": false }, "tuningConfig": { "type": "index", diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json index 544d191d925..832b43b7da7 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json @@ -3,6 +3,14 @@ "spec": { "dataSchema": { "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + {"type": "string", "name": "language", "createBitmapIndex": false} + ] + }, "metricsSpec": [ { "type": "count", @@ -39,27 +47,17 @@ "fieldName": "user" } ], - "granularitySpec": %%GRANULARITYSPEC%%, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - {"type": "string", "name": "language", "createBitmapIndex": false} - ] - } - } - } + "granularitySpec": %%GRANULARITYSPEC%% }, "ioConfig": { "type": "index", - "firehose": { + "inputSource": { "type": "local", "baseDir": "/resources/data/batch_index/json", "filter": "wikipedia_index_data*" + }, + "inputFormat": { + "type": "json" } }, "tuningConfig": { diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_transform.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_transform.json deleted file mode 100644 index 9f3128fb303..00000000000 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_task_with_transform.json +++ /dev/null @@ -1,103 +0,0 @@ -{ - "type": "index", - "spec": { - "dataSchema": { - "dataSource": "%%DATASOURCE%%", - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "triple-added", - "fieldName": "triple-added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - }, - { - "name": "thetaSketch", - "type": "thetaSketch", - "fieldName": "user" - }, - { - "name": "quantilesDoublesSketch", - "type": "quantilesDoublesSketch", - "fieldName": "delta" - }, - { - "name": "HLLSketchBuild", - "type": "HLLSketchBuild", - "fieldName": "user" - } - ], - "granularitySpec": { - "segmentGranularity": "DAY", - "queryGranularity": "second", - "intervals" : [ "2013-08-31/2013-09-02" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } - }, - "transformSpec": { - "transforms": [ - { - "type": "expression", - "name": "language", - "expression": "concat('language-', language)" - }, - { - "type": "expression", - "name": "triple-added", - "expression": "added * 3" - } - ] - } - }, - "ioConfig": { - "type": "index", - "firehose": { - "type": "local", - "baseDir": "/resources/data/batch_index/json", - "filter": "wikipedia_index_data*" - } - }, - "tuningConfig": { - "type": "index", - "maxRowsPerSegment": 3 - } - } -} \ No newline at end of file diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json index 35b115c9f19..3f49cc9f120 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json @@ -3,6 +3,25 @@ "spec": { "dataSchema": { "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, "metricsSpec": [ { "type": "count", @@ -43,38 +62,17 @@ "segmentGranularity": "DAY", "queryGranularity": "second", "intervals" : [ "2013-08-31/2013-09-02" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } } }, "ioConfig": { "type": "index", - "firehose": { + "inputSource": { "type": "local", "baseDir": "/resources/data/batch_index/json", "filter": "wikipedia_index_data*" + }, + "inputFormat": { + "type": "json" } }, "tuningConfig": { diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_task.json index 43264a8c675..36bb6a9c27a 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_task.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_index_task.json @@ -3,6 +3,14 @@ "spec": { "dataSchema": { "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "continent" + ] + }, "metricsSpec": [ { "type": "count", @@ -38,27 +46,17 @@ "segmentGranularity": "DAY", "queryGranularity": "DAY", "intervals" : [ "2013-08-31/2013-09-02" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "continent" - ] - } - } } }, "ioConfig": { "type": "index", - "firehose": { + "inputSource": { "type": "local", "baseDir": "/resources/data/batch_index/json", "filter": "wikipedia_index_data*" + }, + "inputFormat": { + "type": "json" } }, "tuningConfig": { diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_task.json index 127461dd117..4a8b60e9466 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_task.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_merge_reindex_task.json @@ -3,6 +3,15 @@ "spec": { "dataSchema": { "dataSource": "%%REINDEX_DATASOURCE%%", + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "continent" + ] + }, "metricsSpec": [ { "type": "doubleSum", @@ -34,28 +43,17 @@ "segmentGranularity": "DAY", "queryGranularity": "DAY", "intervals" : [ "2013-08-31/2013-09-01" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - "continent" - ] - } - } } }, "ioConfig": { "type": "index", - "firehose": { - "type": "ingestSegment", + "inputSource": { + "type": "druid", "dataSource": "%%DATASOURCE%%", "interval": "2013-08-31/2013-09-01" + }, + "inputFormat": { + "type": "json" } }, "tuningConfig": { diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_task.json index e83b1109da0..ba259fd830f 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_task.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_index_task.json @@ -3,6 +3,25 @@ "spec": { "dataSchema": { "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, "metricsSpec": [ { "type": "count", @@ -28,38 +47,17 @@ "segmentGranularity": "DAY", "queryGranularity": "second", "intervals" : [ "2013-08-31/2013-09-02" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } } }, "ioConfig": { "type": "index_parallel", - "firehose": { + "inputSource": { "type": "local", "baseDir": "/resources/data/batch_index/json", "filter": "wikipedia_index_data*" + }, + "inputFormat": { + "type" : "json" } }, "tuningConfig": { diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json index e995a0f299e..3749923cd20 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json @@ -3,6 +3,15 @@ "spec": { "dataSchema": { "dataSource": "%%REINDEX_DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensionExclusions": [ + "robot", + "continent" + ] + }, "metricsSpec": [ { "type": "count", @@ -30,29 +39,18 @@ "intervals": [ "2013-08-31/2013-09-02" ] - }, - "parser": { - "parseSpec": { - "format": "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensionExclusions": [ - "robot", - "continent" - ] - } - } } }, "ioConfig": { "type": "index_parallel", - "firehose": { - "type": "ingestSegment", + "inputSource": { + "type": "druid", "dataSource": "%%DATASOURCE%%", "interval": "2013-08-31/2013-09-02", "maxInputSegmentBytesPerTask": 1 + }, + "inputFormat": { + "type": "json" } }, "tuningConfig": { diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_task.json index 9ab153c6862..8e3a5bebd45 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_task.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_parallel_reindex_task.json @@ -3,6 +3,25 @@ "spec": { "dataSchema": { "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, "metricsSpec": [ { "type": "count", @@ -27,38 +46,17 @@ "granularitySpec": { "segmentGranularity": "DAY", "queryGranularity": "second" - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } } }, "ioConfig": { "type": "index_parallel", - "firehose": { + "inputSource": { "type": "local", "baseDir": "/resources/data/batch_index/json", "filter": "wikipedia_index_data2*" + }, + "inputFormat": { + "type": "json" } }, "tuningConfig": { diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task.json index e277a9127f4..8a526715f41 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task.json @@ -3,6 +3,13 @@ "spec": { "dataSchema": { "dataSource": "%%REINDEX_DATASOURCE%%", + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensionExclusions" : ["robot", "continent"] + }, "metricsSpec": [ { "type": "doubleSum", @@ -24,26 +31,17 @@ "segmentGranularity": "DAY", "queryGranularity": "second", "intervals" : [ "2013-08-31/2013-09-01" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensionExclusions" : ["robot", "continent"] - } - } } }, "ioConfig": { "type": "index", - "firehose": { - "type": "ingestSegment", + "inputSource": { + "type": "druid", "dataSource": "%%DATASOURCE%%", "interval": "2013-08-31/2013-09-01" + }, + "inputFormat": { + "type": "json" } }, "tuningConfig": { diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json deleted file mode 100644 index 029b136d441..00000000000 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json +++ /dev/null @@ -1,108 +0,0 @@ -{ - "type": "index", - "spec": { - "dataSchema": { - "dataSource": "%%REINDEX_DATASOURCE%%", - "metricsSpec": [ - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "triple-added", - "fieldName": "triple-added" - }, - { - "type": "doubleSum", - "name": "one-plus-triple-added", - "fieldName": "one-plus-triple-added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "double-deleted", - "fieldName": "double-deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - } - ], - "granularitySpec": { - "segmentGranularity": "DAY", - "queryGranularity": "second", - "intervals" : [ "2013-08-31/2013-09-01" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "page", - "newPage", - "anonymous", - "namespace", - "country", - "region", - "city" - ] - }, - "transformSpec": { - "transforms": [ - { - "type": "expression", - "name": "newPage", - "expression": "page" - }, - { - "type": "expression", - "name": "city", - "expression": "concat('city-', city)" - }, - { - "type": "expression", - "name": "one-plus-triple-added", - "expression": "\"triple-added\" + 1" - }, - { - "type": "expression", - "name": "delta", - "expression": "\"delta\" / 2" - }, - { - "type": "expression", - "name": "double-deleted", - "expression": "deleted * 2" - } - ] - } - } - } - }, - "ioConfig": { - "type": "index", - "firehose": { - "type": "ingestSegment", - "dataSource": "%%DATASOURCE%%", - "interval": "2013-08-31/2013-09-01" - } - }, - "tuningConfig": { - "type": "index" - } - } -} diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json index 4f13b70cb72..00a06ee5260 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json @@ -3,6 +3,27 @@ "spec": { "dataSchema": { "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + "timestamp" + ] + }, "metricsSpec": [ { "type": "count", @@ -43,40 +64,22 @@ "segmentGranularity": "DAY", "queryGranularity": "second", "intervals" : [ "2013-08-31/2013-09-02" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city", - "timestamp" - ] - } - } } }, "ioConfig": { "type": "index", - "firehose": { + "inputSource": { "type": "local", - "baseDir": "/resources/data/batch_index/json", - "filter": "wikipedia_index_data*" - } + "files": [ + "/resources/data/batch_index/json/wikipedia_index_data1.json", + "/resources/data/batch_index/json/wikipedia_index_data2.json", + "/resources/data/batch_index/json/wikipedia_index_data3.json" + ] + }, + "inputFormat": { + "type": "json" + }, + "appendToExisting": false }, "tuningConfig": { "type": "index", diff --git a/integration-tests-ex/cases/src/test/resources/indexer/wikiticker_index_task.json b/integration-tests-ex/cases/src/test/resources/indexer/wikiticker_index_task.json index d450c7b9458..9ee4b0ee151 100644 --- a/integration-tests-ex/cases/src/test/resources/indexer/wikiticker_index_task.json +++ b/integration-tests-ex/cases/src/test/resources/indexer/wikiticker_index_task.json @@ -3,6 +3,17 @@ "spec": { "dataSchema": { "dataSource": "%%DATASOURCE%%", + "dimensionsSpec": { + "dimensions": [ + "channel", + "page", + "user" + ] + }, + "timestampSpec": { + "format": "auto", + "column": "time" + }, "granularitySpec": { "type": "uniform", "segmentGranularity": "day", @@ -11,23 +22,6 @@ "2015-09-12/2015-09-13" ] }, - "parser": { - "type": "hadoopyString", - "parseSpec": { - "format": "json", - "dimensionsSpec": { - "dimensions": [ - "channel", - "page", - "user" - ] - }, - "timestampSpec": { - "format": "auto", - "column": "time" - } - } - }, "metricsSpec": [ { "name": "count", @@ -52,10 +46,13 @@ }, "ioConfig": { "type": "index", - "firehose": { + "inputSource": { "type": "local", "baseDir": "/shared/wikiticker-it", "filter": "wikiticker-2015-09-12-sampled.json.gz" + }, + "inputFormat": { + "type": "json" } }, "tuningConfig": { diff --git a/integration-tests-ex/cases/src/test/resources/multi-stage-query/wikipedia_combining_firehose_index_queries.json b/integration-tests-ex/cases/src/test/resources/multi-stage-query/wikipedia_combining_input_source_index_queries.json similarity index 100% rename from integration-tests-ex/cases/src/test/resources/multi-stage-query/wikipedia_combining_firehose_index_queries.json rename to integration-tests-ex/cases/src/test/resources/multi-stage-query/wikipedia_combining_input_source_index_queries.json diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningFirehoseFactoryIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningFirehoseFactoryIndexTest.java deleted file mode 100644 index c1350f55068..00000000000 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningFirehoseFactoryIndexTest.java +++ /dev/null @@ -1,100 +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.druid.tests.indexer; - -import com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.testing.guice.DruidTestModuleFactory; -import org.apache.druid.tests.TestNGGroup; -import org.testng.annotations.Guice; -import org.testng.annotations.Test; - -import java.io.Closeable; -import java.util.function.Function; - -@Test(groups = {TestNGGroup.BATCH_INDEX, TestNGGroup.QUICKSTART_COMPATIBLE}) -@Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITCombiningFirehoseFactoryIndexTest extends AbstractITBatchIndexTest -{ - private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json"; - private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; - private static final String INDEX_DATASOURCE = "wikipedia_index_test"; - - private static final String COMBINING_INDEX_TASK = "/indexer/wikipedia_combining_firehose_index_task.json"; - private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_firehose_index_queries.json"; - private static final String COMBINING_INDEX_DATASOURCE = "wikipedia_comb_index_test"; - - @Test - public void testIndexData() throws Exception - { - try ( - final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); - final Closeable ignored2 = unloader(COMBINING_INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); - ) { - final Function combiningFirehoseSpecTransform = spec -> { - try { - return StringUtils.replace( - spec, - "%%COMBINING_DATASOURCE%%", - INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix() - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - }; - final Function transform = spec -> { - try { - return StringUtils.replace( - spec, - "%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%", - jsonMapper.writeValueAsString("0") - ); - } - catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - }; - - doIndexTest( - INDEX_DATASOURCE, - INDEX_TASK, - transform, - INDEX_QUERIES_RESOURCE, - false, - true, - true, - new Pair<>(false, false) - ); - doIndexTest( - COMBINING_INDEX_DATASOURCE, - COMBINING_INDEX_TASK, - combiningFirehoseSpecTransform, - COMBINING_QUERIES_RESOURCE, - false, - true, - true, - new Pair<>(false, false) - ); - } - } - -} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningInputSourceParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningInputSourceParallelIndexTest.java index ce49b457b0a..7619bcf8b6e 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningInputSourceParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningInputSourceParallelIndexTest.java @@ -41,7 +41,7 @@ public class ITCombiningInputSourceParallelIndexTest extends AbstractITBatchInde private static final String INDEX_DATASOURCE = "wikipedia_index_test"; private static final String COMBINING_INDEX_TASK = "/indexer/wikipedia_combining_input_source_index_parallel_task.json"; - private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_firehose_index_queries.json"; + private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_input_source_index_queries.json"; private static final String COMBINING_INDEX_DATASOURCE = "wikipedia_comb_index_test"; @Test diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITTransformTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITTransformTest.java index e98623dcd89..045450a04ea 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITTransformTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITTransformTest.java @@ -32,12 +32,10 @@ import java.io.IOException; @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITTransformTest extends AbstractITBatchIndexTest { - private static final String INDEX_TASK_WITH_FIREHOSE = "/indexer/wikipedia_index_task_with_transform.json"; private static final String INDEX_TASK_WITH_INPUT_SOURCE = "/indexer/wikipedia_index_task_with_inputsource_transform.json"; private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries_with_transform.json"; private static final String INDEX_DATASOURCE = "wikipedia_index_test"; - private static final String REINDEX_TASK = "/indexer/wikipedia_reindex_task_with_transforms.json"; private static final String REINDEX_TASK_WITH_DRUID_INPUT_SOURCE = "/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json"; private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_reindex_queries_with_transforms.json"; private static final String REINDEX_DATASOURCE = "wikipedia_reindex_test"; @@ -69,53 +67,4 @@ public class ITTransformTest extends AbstractITBatchIndexTest ); } } - - @Test(enabled = false) - public void testIndexAndReIndexUsingIngestSegmentWithTransforms() throws IOException - { - // TODO: re-instate this test when https://github.com/apache/druid/issues/9591 is fixed - // Move the re-index step into testIndexAndReIndexWithTransformSpec for faster tests! - final String reindexDatasource = REINDEX_DATASOURCE + "-testIndexData"; - try ( - final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); - final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix()) - ) { - doIndexTest( - INDEX_DATASOURCE, - INDEX_TASK_WITH_INPUT_SOURCE, - INDEX_QUERIES_RESOURCE, - false, - true, - true, - new Pair<>(false, false) - ); - doReindexTest( - INDEX_DATASOURCE, - reindexDatasource, - REINDEX_TASK, - REINDEX_QUERIES_RESOURCE, - new Pair<>(false, false) - ); - } - } - - @Test(enabled = false) - public void testIndexWithFirehoseAndTransforms() throws IOException - { - // TODO: re-instate this test when https://github.com/apache/druid/issues/9589 is fixed - final String indexDatasource = INDEX_DATASOURCE + "-firehose"; - try ( - final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix()); - ) { - doIndexTest( - indexDatasource, - INDEX_TASK_WITH_FIREHOSE, - INDEX_QUERIES_RESOURCE, - false, - true, - true, - new Pair<>(false, false) - ); - } - } } diff --git a/integration-tests/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json deleted file mode 100644 index 1e7deffc6b5..00000000000 --- a/integration-tests/src/test/resources/indexer/wikipedia_combining_firehose_index_task.json +++ /dev/null @@ -1,95 +0,0 @@ -{ - "type": "index", - "spec": { - "dataSchema": { - "dataSource": "%%DATASOURCE%%", - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - }, - { - "name": "thetaSketch", - "type": "thetaSketch", - "fieldName": "user" - }, - { - "name": "quantilesDoublesSketch", - "type": "quantilesDoublesSketch", - "fieldName": "delta" - }, - { - "name": "HLLSketchBuild", - "type": "HLLSketchBuild", - "fieldName": "user" - } - ], - "granularitySpec": { - "segmentGranularity": "DAY", - "queryGranularity": "second", - "intervals" : [ "2013-08-31/2013-09-02" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } - } - }, - "ioConfig": { - "type": "index", - "firehose": { - "type": "combining", - "delegates": [ - { - "type": "local", - "baseDir": "/resources/indexer", - "filter": "wikipedia_combining_index_data.json" - }, - { - "type": "ingestSegment", - "dataSource": "%%COMBINING_DATASOURCE%%", - "interval": "2013-08-31/2013-09-02" - } - ] - } - }, - "tuningConfig": { - "type": "index", - "maxRowsPerSegment": 3 - } - } -} diff --git a/integration-tests/src/test/resources/indexer/wikipedia_combining_firehose_index_queries.json b/integration-tests/src/test/resources/indexer/wikipedia_combining_input_source_index_queries.json similarity index 100% rename from integration-tests/src/test/resources/indexer/wikipedia_combining_firehose_index_queries.json rename to integration-tests/src/test/resources/indexer/wikipedia_combining_input_source_index_queries.json diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json b/integration-tests/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json index 1fa8b4eba32..35b7e61daae 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_index_task_with_dimension_spec.json @@ -3,6 +3,26 @@ "spec": { "dataSchema": { "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, "metricsSpec": [ { "type": "count", @@ -43,39 +63,22 @@ "segmentGranularity": "DAY", "queryGranularity": "DAY", "intervals" : [ "2013-08-31/2013-09-02" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } } }, "ioConfig": { "type": "index", - "firehose": { + "inputSource": { "type": "local", - "baseDir": "/resources/data/batch_index/json", - "filter": "wikipedia_index_data*" - } + "files": [ + "/resources/data/batch_index/json/wikipedia_index_data1.json", + "/resources/data/batch_index/json/wikipedia_index_data2.json", + "/resources/data/batch_index/json/wikipedia_index_data3.json" + ] + }, + "inputFormat": { + "type": "json" + }, + "appendToExisting": false }, "tuningConfig": { "type": "index", diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json b/integration-tests/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json index 544d191d925..f594792ff56 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_index_task_with_granularity_spec.json @@ -3,6 +3,14 @@ "spec": { "dataSchema": { "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + {"type": "string", "name": "language", "createBitmapIndex": false} + ] + }, "metricsSpec": [ { "type": "count", @@ -39,27 +47,17 @@ "fieldName": "user" } ], - "granularitySpec": %%GRANULARITYSPEC%%, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - {"type": "string", "name": "language", "createBitmapIndex": false} - ] - } - } - } + "granularitySpec": %%GRANULARITYSPEC%% }, "ioConfig": { "type": "index", - "firehose": { + "inputSource": { "type": "local", "baseDir": "/resources/data/batch_index/json", "filter": "wikipedia_index_data*" + }, + "inputFormat": { + "type" : "json" } }, "tuningConfig": { diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_task_with_transform.json b/integration-tests/src/test/resources/indexer/wikipedia_index_task_with_transform.json deleted file mode 100644 index 9f3128fb303..00000000000 --- a/integration-tests/src/test/resources/indexer/wikipedia_index_task_with_transform.json +++ /dev/null @@ -1,103 +0,0 @@ -{ - "type": "index", - "spec": { - "dataSchema": { - "dataSource": "%%DATASOURCE%%", - "metricsSpec": [ - { - "type": "count", - "name": "count" - }, - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "triple-added", - "fieldName": "triple-added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - }, - { - "name": "thetaSketch", - "type": "thetaSketch", - "fieldName": "user" - }, - { - "name": "quantilesDoublesSketch", - "type": "quantilesDoublesSketch", - "fieldName": "delta" - }, - { - "name": "HLLSketchBuild", - "type": "HLLSketchBuild", - "fieldName": "user" - } - ], - "granularitySpec": { - "segmentGranularity": "DAY", - "queryGranularity": "second", - "intervals" : [ "2013-08-31/2013-09-02" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } - }, - "transformSpec": { - "transforms": [ - { - "type": "expression", - "name": "language", - "expression": "concat('language-', language)" - }, - { - "type": "expression", - "name": "triple-added", - "expression": "added * 3" - } - ] - } - }, - "ioConfig": { - "type": "index", - "firehose": { - "type": "local", - "baseDir": "/resources/data/batch_index/json", - "filter": "wikipedia_index_data*" - } - }, - "tuningConfig": { - "type": "index", - "maxRowsPerSegment": 3 - } - } -} \ No newline at end of file diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json b/integration-tests/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json index 35b115c9f19..49957a512d4 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_index_with_merge_column_limit_task.json @@ -3,6 +3,25 @@ "spec": { "dataSchema": { "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, "metricsSpec": [ { "type": "count", @@ -43,38 +62,17 @@ "segmentGranularity": "DAY", "queryGranularity": "second", "intervals" : [ "2013-08-31/2013-09-02" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } } }, "ioConfig": { "type": "index", - "firehose": { + "inputSource": { "type": "local", "baseDir": "/resources/data/batch_index/json", "filter": "wikipedia_index_data*" + }, + "inputFormat": { + "type" : "json" } }, "tuningConfig": { diff --git a/integration-tests/src/test/resources/indexer/wikipedia_merge_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_merge_index_task.json index 43264a8c675..268a3aef4a8 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_merge_index_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_merge_index_task.json @@ -3,6 +3,14 @@ "spec": { "dataSchema": { "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "continent" + ] + }, "metricsSpec": [ { "type": "count", @@ -38,27 +46,17 @@ "segmentGranularity": "DAY", "queryGranularity": "DAY", "intervals" : [ "2013-08-31/2013-09-02" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "continent" - ] - } - } } }, "ioConfig": { "type": "index", - "firehose": { + "inputSource": { "type": "local", "baseDir": "/resources/data/batch_index/json", "filter": "wikipedia_index_data*" + }, + "inputFormat": { + "type" : "json" } }, "tuningConfig": { diff --git a/integration-tests/src/test/resources/indexer/wikipedia_merge_reindex_task.json b/integration-tests/src/test/resources/indexer/wikipedia_merge_reindex_task.json index 127461dd117..040fff005ce 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_merge_reindex_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_merge_reindex_task.json @@ -3,6 +3,15 @@ "spec": { "dataSchema": { "dataSource": "%%REINDEX_DATASOURCE%%", + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "continent" + ] + }, "metricsSpec": [ { "type": "doubleSum", @@ -34,28 +43,17 @@ "segmentGranularity": "DAY", "queryGranularity": "DAY", "intervals" : [ "2013-08-31/2013-09-01" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - "continent" - ] - } - } } }, "ioConfig": { "type": "index", - "firehose": { - "type": "ingestSegment", + "inputSource": { + "type": "druid", "dataSource": "%%DATASOURCE%%", "interval": "2013-08-31/2013-09-01" + }, + "inputFormat": { + "type" : "json" } }, "tuningConfig": { diff --git a/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json index e83b1109da0..ba259fd830f 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json @@ -3,6 +3,25 @@ "spec": { "dataSchema": { "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, "metricsSpec": [ { "type": "count", @@ -28,38 +47,17 @@ "segmentGranularity": "DAY", "queryGranularity": "second", "intervals" : [ "2013-08-31/2013-09-02" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } } }, "ioConfig": { "type": "index_parallel", - "firehose": { + "inputSource": { "type": "local", "baseDir": "/resources/data/batch_index/json", "filter": "wikipedia_index_data*" + }, + "inputFormat": { + "type" : "json" } }, "tuningConfig": { diff --git a/integration-tests/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json index e995a0f299e..3749923cd20 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_parallel_ingest_segment_index_task.json @@ -3,6 +3,15 @@ "spec": { "dataSchema": { "dataSource": "%%REINDEX_DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensionExclusions": [ + "robot", + "continent" + ] + }, "metricsSpec": [ { "type": "count", @@ -30,29 +39,18 @@ "intervals": [ "2013-08-31/2013-09-02" ] - }, - "parser": { - "parseSpec": { - "format": "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensionExclusions": [ - "robot", - "continent" - ] - } - } } }, "ioConfig": { "type": "index_parallel", - "firehose": { - "type": "ingestSegment", + "inputSource": { + "type": "druid", "dataSource": "%%DATASOURCE%%", "interval": "2013-08-31/2013-09-02", "maxInputSegmentBytesPerTask": 1 + }, + "inputFormat": { + "type": "json" } }, "tuningConfig": { diff --git a/integration-tests/src/test/resources/indexer/wikipedia_parallel_reindex_task.json b/integration-tests/src/test/resources/indexer/wikipedia_parallel_reindex_task.json index 9ab153c6862..21d064843b3 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_parallel_reindex_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_parallel_reindex_task.json @@ -3,6 +3,25 @@ "spec": { "dataSchema": { "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + }, "metricsSpec": [ { "type": "count", @@ -27,38 +46,17 @@ "granularitySpec": { "segmentGranularity": "DAY", "queryGranularity": "second" - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city" - ] - } - } } }, "ioConfig": { "type": "index_parallel", - "firehose": { + "inputSource": { "type": "local", "baseDir": "/resources/data/batch_index/json", "filter": "wikipedia_index_data2*" + }, + "inputFormat": { + "type" : "json" } }, "tuningConfig": { diff --git a/integration-tests/src/test/resources/indexer/wikipedia_reindex_task.json b/integration-tests/src/test/resources/indexer/wikipedia_reindex_task.json index e277a9127f4..861f08ab81b 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_reindex_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_reindex_task.json @@ -3,6 +3,13 @@ "spec": { "dataSchema": { "dataSource": "%%REINDEX_DATASOURCE%%", + "timestampSpec": { + "column": "timestamp", + "format": "iso" + }, + "dimensionsSpec": { + "dimensionExclusions" : ["robot", "continent"] + }, "metricsSpec": [ { "type": "doubleSum", @@ -24,26 +31,17 @@ "segmentGranularity": "DAY", "queryGranularity": "second", "intervals" : [ "2013-08-31/2013-09-01" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensionExclusions" : ["robot", "continent"] - } - } } }, "ioConfig": { "type": "index", - "firehose": { - "type": "ingestSegment", + "inputSource": { + "type": "druid", "dataSource": "%%DATASOURCE%%", "interval": "2013-08-31/2013-09-01" + }, + "inputFormat": { + "type" : "json" } }, "tuningConfig": { diff --git a/integration-tests/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json b/integration-tests/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json deleted file mode 100644 index 029b136d441..00000000000 --- a/integration-tests/src/test/resources/indexer/wikipedia_reindex_task_with_transforms.json +++ /dev/null @@ -1,108 +0,0 @@ -{ - "type": "index", - "spec": { - "dataSchema": { - "dataSource": "%%REINDEX_DATASOURCE%%", - "metricsSpec": [ - { - "type": "doubleSum", - "name": "added", - "fieldName": "added" - }, - { - "type": "doubleSum", - "name": "triple-added", - "fieldName": "triple-added" - }, - { - "type": "doubleSum", - "name": "one-plus-triple-added", - "fieldName": "one-plus-triple-added" - }, - { - "type": "doubleSum", - "name": "deleted", - "fieldName": "deleted" - }, - { - "type": "doubleSum", - "name": "double-deleted", - "fieldName": "double-deleted" - }, - { - "type": "doubleSum", - "name": "delta", - "fieldName": "delta" - } - ], - "granularitySpec": { - "segmentGranularity": "DAY", - "queryGranularity": "second", - "intervals" : [ "2013-08-31/2013-09-01" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp", - "format": "iso" - }, - "dimensionsSpec": { - "dimensions": [ - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "page", - "newPage", - "anonymous", - "namespace", - "country", - "region", - "city" - ] - }, - "transformSpec": { - "transforms": [ - { - "type": "expression", - "name": "newPage", - "expression": "page" - }, - { - "type": "expression", - "name": "city", - "expression": "concat('city-', city)" - }, - { - "type": "expression", - "name": "one-plus-triple-added", - "expression": "\"triple-added\" + 1" - }, - { - "type": "expression", - "name": "delta", - "expression": "\"delta\" / 2" - }, - { - "type": "expression", - "name": "double-deleted", - "expression": "deleted * 2" - } - ] - } - } - } - }, - "ioConfig": { - "type": "index", - "firehose": { - "type": "ingestSegment", - "dataSource": "%%DATASOURCE%%", - "interval": "2013-08-31/2013-09-01" - } - }, - "tuningConfig": { - "type": "index" - } - } -} diff --git a/integration-tests/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json index 4f13b70cb72..00a06ee5260 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_with_timestamp_index_task.json @@ -3,6 +3,27 @@ "spec": { "dataSchema": { "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city", + "timestamp" + ] + }, "metricsSpec": [ { "type": "count", @@ -43,40 +64,22 @@ "segmentGranularity": "DAY", "queryGranularity": "second", "intervals" : [ "2013-08-31/2013-09-02" ] - }, - "parser": { - "parseSpec": { - "format" : "json", - "timestampSpec": { - "column": "timestamp" - }, - "dimensionsSpec": { - "dimensions": [ - "page", - {"type": "string", "name": "language", "createBitmapIndex": false}, - "user", - "unpatrolled", - "newPage", - "robot", - "anonymous", - "namespace", - "continent", - "country", - "region", - "city", - "timestamp" - ] - } - } } }, "ioConfig": { "type": "index", - "firehose": { + "inputSource": { "type": "local", - "baseDir": "/resources/data/batch_index/json", - "filter": "wikipedia_index_data*" - } + "files": [ + "/resources/data/batch_index/json/wikipedia_index_data1.json", + "/resources/data/batch_index/json/wikipedia_index_data2.json", + "/resources/data/batch_index/json/wikipedia_index_data3.json" + ] + }, + "inputFormat": { + "type": "json" + }, + "appendToExisting": false }, "tuningConfig": { "type": "index", diff --git a/integration-tests/src/test/resources/indexer/wikiticker_index_task.json b/integration-tests/src/test/resources/indexer/wikiticker_index_task.json index d450c7b9458..9ee4b0ee151 100644 --- a/integration-tests/src/test/resources/indexer/wikiticker_index_task.json +++ b/integration-tests/src/test/resources/indexer/wikiticker_index_task.json @@ -3,6 +3,17 @@ "spec": { "dataSchema": { "dataSource": "%%DATASOURCE%%", + "dimensionsSpec": { + "dimensions": [ + "channel", + "page", + "user" + ] + }, + "timestampSpec": { + "format": "auto", + "column": "time" + }, "granularitySpec": { "type": "uniform", "segmentGranularity": "day", @@ -11,23 +22,6 @@ "2015-09-12/2015-09-13" ] }, - "parser": { - "type": "hadoopyString", - "parseSpec": { - "format": "json", - "dimensionsSpec": { - "dimensions": [ - "channel", - "page", - "user" - ] - }, - "timestampSpec": { - "format": "auto", - "column": "time" - } - } - }, "metricsSpec": [ { "name": "count", @@ -52,10 +46,13 @@ }, "ioConfig": { "type": "index", - "firehose": { + "inputSource": { "type": "local", "baseDir": "/shared/wikiticker-it", "filter": "wikiticker-2015-09-12-sampled.json.gz" + }, + "inputFormat": { + "type": "json" } }, "tuningConfig": { diff --git a/processing/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java b/processing/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java deleted file mode 100644 index d63ac3b6963..00000000000 --- a/processing/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java +++ /dev/null @@ -1,72 +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.druid.data.input; - -import com.fasterxml.jackson.annotation.JsonIgnore; -import org.apache.druid.data.input.impl.InputRowParser; - -import javax.annotation.Nullable; -import java.io.IOException; -import java.util.stream.Stream; - -/** - * {@link FiniteFirehoseFactory} designed for batch processing. Its implementations assume that the amount of inputs is - * limited. - * - * This class is deprecated in favor of {@link InputSource}. - * - * @param parser type - * @param input split type - */ -@Deprecated -public interface FiniteFirehoseFactory extends FirehoseFactory -{ - /** - * Returns true if this {@link FiniteFirehoseFactory} supports parallel batch indexing. - */ - @JsonIgnore - @Override - default boolean isSplittable() - { - return true; - } - - /** - * Returns a {@link Stream} for {@link InputSplit}s. In parallel batch indexing, each {@link InputSplit} is processed - * by a sub task. - * - * Listing splits may cause high overhead in some implementations. In this case, {@link InputSplit}s should be listed - * lazily so that the listing overhead could be amortized. - */ - @JsonIgnore - Stream> getSplits(@Nullable SplitHintSpec splitHintSpec) throws IOException; - - /** - * Returns number of splits returned by {@link #getSplits}. - */ - @JsonIgnore - int getNumSplits(@Nullable SplitHintSpec splitHintSpec) throws IOException; - - /** - * Returns the same {@link FiniteFirehoseFactory} but with the given {@link InputSplit}. The returned - * {@link FiniteFirehoseFactory} is used by sub tasks in parallel batch indexing. - */ - FiniteFirehoseFactory withSplit(InputSplit split); -} diff --git a/processing/src/main/java/org/apache/druid/data/input/Firehose.java b/processing/src/main/java/org/apache/druid/data/input/Firehose.java index 66995bda184..a9ff6f33159 100644 --- a/processing/src/main/java/org/apache/druid/data/input/Firehose.java +++ b/processing/src/main/java/org/apache/druid/data/input/Firehose.java @@ -43,6 +43,7 @@ import java.io.IOException; * and {@link #nextRow()}. *

*/ +@Deprecated @ExtensionPoint public interface Firehose extends Closeable { diff --git a/processing/src/main/java/org/apache/druid/data/input/FirehoseFactory.java b/processing/src/main/java/org/apache/druid/data/input/FirehoseFactory.java index 5561dc93372..6ec8657d435 100644 --- a/processing/src/main/java/org/apache/druid/data/input/FirehoseFactory.java +++ b/processing/src/main/java/org/apache/druid/data/input/FirehoseFactory.java @@ -22,7 +22,6 @@ package org.apache.druid.data.input; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; import org.apache.druid.guice.annotations.ExtensionPoint; import org.apache.druid.java.util.common.parsers.ParseException; @@ -34,7 +33,10 @@ import java.io.IOException; * FirehoseFactory creates a {@link Firehose} which is an interface holding onto the stream of incoming data. * It currently provides two methods for creating a {@link Firehose} and their default implementations call each other * for the backward compatibility. Implementations of this interface must implement one of these methods. + * + * This class is deprecated in favor of {@link InputSource} */ +@Deprecated @ExtensionPoint @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") public interface FirehoseFactory @@ -62,9 +64,6 @@ public interface FirehoseFactory * If this method returns null, then any attempt to call hasMore(), nextRow() and close() on the return * value will throw a surprising NPE. Throwing IOException on connection failure or runtime exception on * invalid configuration is preferred over returning null. - *

- * Some fire hoses like {@link PrefetchableTextFilesFirehoseFactory} may use a temporary - * directory to cache data in it. * * @param parser an input row parser * @param temporaryDirectory a directory where temporary files are stored @@ -74,18 +73,6 @@ public interface FirehoseFactory return connect(parser); } - /** - * Initialization method that connects up the firehose. This method is intended for use by the sampler, and allows - * implementors to return a more efficient firehose, knowing that only a small number of rows will be read. - * - * @param parser an input row parser - * @param temporaryDirectory a directory where temporary files are stored - */ - default Firehose connectForSampler(T parser, @Nullable File temporaryDirectory) throws IOException, ParseException - { - return connect(parser, temporaryDirectory); - } - @SuppressWarnings("unused") @JsonIgnore default boolean isSplittable() diff --git a/processing/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java b/processing/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java deleted file mode 100644 index ca74798e96f..00000000000 --- a/processing/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java +++ /dev/null @@ -1,99 +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.druid.data.input; - -import com.google.common.base.Preconditions; -import org.apache.druid.data.input.impl.FirehoseToInputSourceReaderAdaptor; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.SplittableInputSource; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.util.stream.Stream; - -public class FirehoseFactoryToInputSourceAdaptor extends AbstractInputSource implements SplittableInputSource -{ - private final FiniteFirehoseFactory firehoseFactory; - private final InputRowParser inputRowParser; - - public FirehoseFactoryToInputSourceAdaptor(FiniteFirehoseFactory firehoseFactory, InputRowParser inputRowParser) - { - this.firehoseFactory = firehoseFactory; - this.inputRowParser = Preconditions.checkNotNull(inputRowParser, "inputRowParser"); - } - - public FiniteFirehoseFactory getFirehoseFactory() - { - return firehoseFactory; - } - - @Override - public boolean isSplittable() - { - return firehoseFactory.isSplittable(); - } - - @Override - public Stream createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) - throws IOException - { - if (firehoseFactory.isSplittable()) { - return firehoseFactory.getSplits(splitHintSpec); - } else { - throw new UnsupportedOperationException(); - } - } - - @Override - public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) throws IOException - { - if (firehoseFactory.isSplittable()) { - return firehoseFactory.getNumSplits(splitHintSpec); - } else { - throw new UnsupportedOperationException(); - } - } - - @Override - public SplittableInputSource withSplit(InputSplit split) - { - if (firehoseFactory.isSplittable()) { - return new FirehoseFactoryToInputSourceAdaptor( - firehoseFactory.withSplit(split), - inputRowParser - ); - } else { - throw new UnsupportedOperationException(); - } - } - - @Override - public boolean needsFormat() - { - return false; - } - - @Override - protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) - { - return new FirehoseToInputSourceReaderAdaptor(firehoseFactory, inputRowParser, temporaryDirectory); - } -} diff --git a/processing/src/main/java/org/apache/druid/data/input/SplitHintSpec.java b/processing/src/main/java/org/apache/druid/data/input/SplitHintSpec.java index 3c40ff840a7..100dfc8443e 100644 --- a/processing/src/main/java/org/apache/druid/data/input/SplitHintSpec.java +++ b/processing/src/main/java/org/apache/druid/data/input/SplitHintSpec.java @@ -34,8 +34,6 @@ import java.util.function.Function; * want to give some hints to control the amount of data each sub task will read. SplitHintSpec can be used for this * purpose. Implementations can ignore the given hint. * - * @see FiniteFirehoseFactory#getSplits(SplitHintSpec) - * @see FiniteFirehoseFactory#getNumSplits(SplitHintSpec) * @see SplittableInputSource#createSplits * @see SplittableInputSource#estimateNumSplits */ diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java b/processing/src/main/java/org/apache/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java deleted file mode 100644 index 592d014a6dc..00000000000 --- a/processing/src/main/java/org/apache/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java +++ /dev/null @@ -1,144 +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.druid.data.input.impl; - -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import org.apache.commons.io.IOUtils; -import org.apache.commons.io.LineIterator; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.SplitHintSpec; -import org.apache.druid.java.util.common.logger.Logger; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.nio.charset.StandardCharsets; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.stream.Stream; - -/** - * This is an abstract class for firehose factory for making firehoses reading text files. - * It provides an unified {@link #connect(StringInputRowParser, File)} implementation for its subclasses. - * - * @param object type representing input data - */ -public abstract class AbstractTextFilesFirehoseFactory - implements FiniteFirehoseFactory -{ - private static final Logger LOG = new Logger(AbstractTextFilesFirehoseFactory.class); - - private List objects; - - @Override - public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirectory) throws IOException - { - initializeObjectsIfNeeded(); - final Iterator iterator = objects.iterator(); - return new FileIteratingFirehose( - new Iterator() - { - @Override - public boolean hasNext() - { - return iterator.hasNext(); - } - - @Override - public LineIterator next() - { - if (!hasNext()) { - throw new NoSuchElementException(); - } - final T object = iterator.next(); - try { - return IOUtils.lineIterator(wrapObjectStream(object, openObjectStream(object)), StandardCharsets.UTF_8); - } - catch (Exception e) { - LOG.error(e, "Exception reading object[%s]", object); - throw new RuntimeException(e); - } - } - }, - firehoseParser - ); - } - - protected void initializeObjectsIfNeeded() throws IOException - { - if (objects == null) { - objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "initObjects")); - } - } - - public List getObjects() - { - return objects; - } - - @Override - public Stream> getSplits(@Nullable SplitHintSpec splitHintSpec) throws IOException - { - initializeObjectsIfNeeded(); - return getObjects().stream().map(InputSplit::new); - } - - @Override - public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) throws IOException - { - initializeObjectsIfNeeded(); - return getObjects().size(); - } - - /** - * Initialize objects to be read by this firehose. Since firehose factories are constructed whenever - * org.apache.druid.indexing.common.task.Task objects are deserialized, actual initialization of objects is deferred - * until {@link #connect(StringInputRowParser, File)} is called. - * - * @return a collection of initialized objects. - */ - protected abstract Collection initObjects() throws IOException; - - /** - * Open an input stream from the given object. If the object is compressed, this method should return a byte stream - * as it is compressed. The object compression should be handled in {@link #wrapObjectStream(Object, InputStream)}. - * - * @param object an object to be read - * - * @return an input stream for the object - */ - protected abstract InputStream openObjectStream(T object) throws IOException; - - /** - * Wrap the given input stream if needed. The decompression logic should be applied to the given stream if the object - * is compressed. - * - * @param object an input object - * @param stream a stream for the object - * @return an wrapped input stream - */ - protected abstract InputStream wrapObjectStream(T object, InputStream stream) throws IOException; -} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/FileIteratingFirehose.java b/processing/src/main/java/org/apache/druid/data/input/impl/FileIteratingFirehose.java deleted file mode 100644 index 42c627df213..00000000000 --- a/processing/src/main/java/org/apache/druid/data/input/impl/FileIteratingFirehose.java +++ /dev/null @@ -1,118 +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.druid.data.input.impl; - -import org.apache.commons.io.LineIterator; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusRawValues; -import org.apache.druid.java.util.common.parsers.ParseException; - -import javax.annotation.Nullable; -import java.io.Closeable; -import java.io.IOException; -import java.util.Iterator; -import java.util.NoSuchElementException; - -public class FileIteratingFirehose implements Firehose -{ - private final Iterator lineIterators; - private final StringInputRowParser parser; - - private LineIterator lineIterator = null; - - private final Closeable closer; - - public FileIteratingFirehose( - Iterator lineIterators, - StringInputRowParser parser - ) - { - this(lineIterators, parser, null); - } - - public FileIteratingFirehose( - Iterator lineIterators, - StringInputRowParser parser, - Closeable closer - ) - { - this.lineIterators = lineIterators; - this.parser = parser; - this.closer = closer; - } - - @Override - public boolean hasMore() throws IOException - { - while ((lineIterator == null || !lineIterator.hasNext()) && lineIterators.hasNext()) { - lineIterator = getNextLineIterator(); - } - - return lineIterator != null && lineIterator.hasNext(); - } - - @Nullable - @Override - public InputRow nextRow() throws IOException - { - if (!hasMore()) { - throw new NoSuchElementException(); - } - - return parser.parse(lineIterator.next()); - } - - @Override - public InputRowListPlusRawValues nextRowWithRaw() throws IOException - { - if (!hasMore()) { - throw new NoSuchElementException(); - } - - String raw = lineIterator.next(); - try { - return InputRowListPlusRawValues.of(parser.parse(raw), parser.parseString(raw)); - } - catch (ParseException e) { - return InputRowListPlusRawValues.of(parser.parseString(raw), e); - } - } - - private LineIterator getNextLineIterator() throws IOException - { - if (lineIterator != null) { - lineIterator.close(); - } - - final LineIterator iterator = lineIterators.next(); - parser.startFileFromBeginning(); - return iterator; - } - - @Override - public void close() throws IOException - { - try (Closeable ignore = closer; - Closeable ignore2 = lineIterator) { - // close both via try-with-resources - } - } -} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java b/processing/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java deleted file mode 100644 index 778ea3bef30..00000000000 --- a/processing/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java +++ /dev/null @@ -1,124 +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.druid.data.input.impl; - -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusRawValues; -import org.apache.druid.data.input.InputSourceReader; -import org.apache.druid.data.input.InputStats; -import org.apache.druid.java.util.common.parsers.CloseableIterator; - -import java.io.File; -import java.io.IOException; -import java.io.UncheckedIOException; - -public class FirehoseToInputSourceReaderAdaptor implements InputSourceReader -{ - private final FirehoseFactory firehoseFactory; - private final InputRowParser inputRowParser; - private final File temporaryDirectory; - - public FirehoseToInputSourceReaderAdaptor( - FirehoseFactory firehoseFactory, - InputRowParser inputRowParser, - File temporaryDirectory - ) - { - this.firehoseFactory = firehoseFactory; - this.inputRowParser = inputRowParser; - this.temporaryDirectory = temporaryDirectory; - } - - @Override - public CloseableIterator read(InputStats inputStats) throws IOException - { - return new CloseableIterator() - { - final Firehose firehose = firehoseFactory.connect(inputRowParser, temporaryDirectory); - - @Override - public boolean hasNext() - { - try { - return firehose.hasMore(); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - @Override - public InputRow next() - { - try { - return firehose.nextRow(); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - @Override - public void close() throws IOException - { - firehose.close(); - } - }; - } - - @Override - public CloseableIterator sample() throws IOException - { - return new CloseableIterator() - { - final Firehose firehose = firehoseFactory.connectForSampler(inputRowParser, temporaryDirectory); - - @Override - public boolean hasNext() - { - try { - return firehose.hasMore(); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - @Override - public InputRowListPlusRawValues next() - { - try { - return firehose.nextRowWithRaw(); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - @Override - public void close() throws IOException - { - firehose.close(); - } - }; - } -} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/RetryingInputStream.java b/processing/src/main/java/org/apache/druid/data/input/impl/RetryingInputStream.java index f4693fc8154..d517d5fcb1b 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/RetryingInputStream.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/RetryingInputStream.java @@ -24,7 +24,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.base.Throwables; import com.google.common.io.CountingInputStream; -import org.apache.druid.data.input.impl.prefetch.Fetcher; import org.apache.druid.data.input.impl.prefetch.ObjectOpenFunction; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.RetryUtils; @@ -36,7 +35,7 @@ import java.io.IOException; import java.io.InputStream; /** - * This class is used by {@link Fetcher} when prefetch is disabled. It's responsible for re-opening the underlying input + * This class is responsible for re-opening the underlying input * stream for the input object on the given {@link #retryCondition}. * * @param object type diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/CacheManager.java b/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/CacheManager.java deleted file mode 100644 index ba8f7db9918..00000000000 --- a/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/CacheManager.java +++ /dev/null @@ -1,97 +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.druid.data.input.impl.prefetch; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.logger.Logger; - -import java.util.ArrayList; -import java.util.List; - -/** - * A class managing cached files used by {@link PrefetchableTextFilesFirehoseFactory}. - */ -public class CacheManager -{ - private static final Logger LOG = new Logger(CacheManager.class); - - // A roughly max size of total cached objects which means the actual cached size can be bigger. The reason is our - // current client implementations for cloud storages like s3 don't support range scan yet, so we must download the - // whole file at once. It's still possible for the size of cached data to not exceed these variables by estimating the - // after-fetch size, but it makes us to consider the case when any files cannot be fetched due to their large size, - // which makes the implementation complicated. - private final long maxCacheCapacityBytes; - - private final List> files = new ArrayList<>(); - - private long totalCachedBytes; - - public CacheManager(long maxCacheCapacityBytes) - { - this.maxCacheCapacityBytes = maxCacheCapacityBytes; - } - - public boolean isEnabled() - { - return maxCacheCapacityBytes > 0; - } - - public boolean cacheable() - { - // maxCacheCapacityBytes is a rough limit, so if totalCachedBytes is larger than it, no more caching is - // allowed. - return totalCachedBytes < maxCacheCapacityBytes; - } - - FetchedFile cache(FetchedFile fetchedFile) - { - if (!cacheable()) { - throw new ISE( - "Cache space is full. totalCachedBytes[%d], maxCacheCapacityBytes[%d]", - totalCachedBytes, - maxCacheCapacityBytes - ); - } - - final FetchedFile cachedFile = fetchedFile.cache(); - files.add(cachedFile); - totalCachedBytes += cachedFile.length(); - - LOG.info("Object[%s] is cached. Current cached bytes is [%d]", cachedFile.getObject(), totalCachedBytes); - return cachedFile; - } - - List> getFiles() - { - return files; - } - - @VisibleForTesting - long getTotalCachedBytes() - { - return totalCachedBytes; - } - - public long getMaxCacheCapacityBytes() - { - return maxCacheCapacityBytes; - } -} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/FetchConfig.java b/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/FetchConfig.java deleted file mode 100644 index c645d73bead..00000000000 --- a/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/FetchConfig.java +++ /dev/null @@ -1,96 +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.druid.data.input.impl.prefetch; - -import javax.annotation.Nullable; -import java.util.concurrent.TimeUnit; - -/** - * Holds configurations required by {@link Fetcher} for fetching objects. - */ -public class FetchConfig -{ - private static final long DEFAULT_MAX_CACHE_CAPACITY_BYTES = 1024 * 1024 * 1024; // 1GiB - private static final long DEFAULT_MAX_FETCH_CAPACITY_BYTES = 1024 * 1024 * 1024; // 1GiB - private static final long DEFAULT_FETCH_TIMEOUT_MS = TimeUnit.SECONDS.toMillis(60); - private static final int DEFAULT_MAX_FETCH_RETRY = 3; - - // A roughly max size of total fetched objects, but the actual fetched size can be bigger. The reason is our current - // client implementations for cloud storages like s3 don't support range scan yet, so we must download the whole file - // at once. It's still possible for the size of cached/fetched data to not exceed these variables by estimating the - // after-fetch size, but it makes us consider the case when any files cannot be fetched due to their large size, which - // makes the implementation complicated. - private final long maxFetchCapacityBytes; - - private final long maxCacheCapacityBytes; - - private final long prefetchTriggerBytes; - - // timeout for fetching an object from the remote site - private final long fetchTimeout; - - private final int maxFetchRetry; - - public FetchConfig( - @Nullable Long maxCacheCapacityBytes, - @Nullable Long maxFetchCapacityBytes, - @Nullable Long prefetchTriggerBytes, - @Nullable Long fetchTimeout, - @Nullable Integer maxFetchRetry - ) - { - this.maxCacheCapacityBytes = maxCacheCapacityBytes == null - ? DEFAULT_MAX_CACHE_CAPACITY_BYTES - : maxCacheCapacityBytes; - this.maxFetchCapacityBytes = maxFetchCapacityBytes == null - ? DEFAULT_MAX_FETCH_CAPACITY_BYTES - : maxFetchCapacityBytes; - this.prefetchTriggerBytes = prefetchTriggerBytes == null - ? this.maxFetchCapacityBytes / 2 - : prefetchTriggerBytes; - this.fetchTimeout = fetchTimeout == null ? DEFAULT_FETCH_TIMEOUT_MS : fetchTimeout; - this.maxFetchRetry = maxFetchRetry == null ? DEFAULT_MAX_FETCH_RETRY : maxFetchRetry; - } - - public long getMaxCacheCapacityBytes() - { - return maxCacheCapacityBytes; - } - - public long getMaxFetchCapacityBytes() - { - return maxFetchCapacityBytes; - } - - public long getPrefetchTriggerBytes() - { - return prefetchTriggerBytes; - } - - public long getFetchTimeout() - { - return fetchTimeout; - } - - public int getMaxFetchRetry() - { - return maxFetchRetry; - } -} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/FetchedFile.java b/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/FetchedFile.java deleted file mode 100644 index 9874820e072..00000000000 --- a/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/FetchedFile.java +++ /dev/null @@ -1,68 +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.druid.data.input.impl.prefetch; - -import java.io.Closeable; -import java.io.File; - -/** - * A class containing meta information about fetched objects. This class used by {@link Fetcher}. - */ -class FetchedFile -{ - // Original object - private final T object; - // Fetched file stored in local disk - private final File file; - // Closer which is called when the file is not needed anymore. Usually this deletes the file except for cached files. - private final Closeable resourceCloser; - - FetchedFile(T object, File file, Closeable resourceCloser) - { - this.object = object; - this.file = file; - this.resourceCloser = resourceCloser; - } - - long length() - { - return file.length(); - } - - T getObject() - { - return object; - } - - File getFile() - { - return file; - } - - Closeable getResourceCloser() - { - return resourceCloser; - } - - FetchedFile cache() - { - return new FetchedFile<>(object, file, () -> {}); - } -} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/Fetcher.java b/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/Fetcher.java deleted file mode 100644 index 436f4283343..00000000000 --- a/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/Fetcher.java +++ /dev/null @@ -1,292 +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.druid.data.input.impl.prefetch; - -import com.google.common.base.Preconditions; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.logger.Logger; - -import javax.annotation.Nullable; -import java.io.Closeable; -import java.io.File; -import java.io.IOException; -import java.util.ArrayDeque; -import java.util.Deque; -import java.util.Iterator; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicLong; - -/** - * A file fetcher used by {@link PrefetchableTextFilesFirehoseFactory} and PrefetchSqlFirehoseFactory (in druid-server). - * See the javadoc of {@link PrefetchableTextFilesFirehoseFactory} for more details. - */ -public abstract class Fetcher implements Iterator> -{ - private static final Logger LOG = new Logger(Fetcher.class); - private static final String FETCH_FILE_PREFIX = "fetch-"; - private final CacheManager cacheManager; - private final List objects; - private final ExecutorService fetchExecutor; - - @Nullable - private final File temporaryDirectory; - - private final boolean prefetchEnabled; - - private final LinkedBlockingQueue> fetchedFiles = new LinkedBlockingQueue<>(); - - // Number of bytes of current fetched files. - // This is updated when a file is successfully fetched, a fetched file is deleted, or a fetched file is - // cached. - private final AtomicLong fetchedBytes = new AtomicLong(0); - private final Deque> fetchFutures = new ArrayDeque<>(); - private final FetchConfig fetchConfig; - - // nextFetchIndex indicates which object should be downloaded when fetch is triggered. - // This variable is always read by the same thread regardless of prefetch is enabled or not. - private int nextFetchIndex; - - private int numRemainingObjects; - - public Fetcher( - CacheManager cacheManager, - List objects, - ExecutorService fetchExecutor, - @Nullable File temporaryDirectory, - FetchConfig fetchConfig - ) - { - this.cacheManager = cacheManager; - this.objects = objects; - this.fetchExecutor = fetchExecutor; - this.temporaryDirectory = temporaryDirectory; - this.fetchConfig = fetchConfig; - this.prefetchEnabled = fetchConfig.getMaxFetchCapacityBytes() > 0; - this.numRemainingObjects = objects.size(); - - // (*) If cache is initialized, put all cached files to the queue. - this.fetchedFiles.addAll(cacheManager.getFiles()); - this.nextFetchIndex = fetchedFiles.size(); - if (cacheManager.isEnabled() || prefetchEnabled) { - Preconditions.checkNotNull(temporaryDirectory, "temporaryDirectory"); - } - if (prefetchEnabled) { - fetchIfNeeded(0L); - } - } - - /** - * Submit a fetch task if remainingBytes is smaller than prefetchTriggerBytes. - */ - private void fetchIfNeeded(long remainingBytes) - { - if ((fetchFutures.isEmpty() || fetchFutures.peekLast().isDone()) - && remainingBytes <= fetchConfig.getPrefetchTriggerBytes()) { - Future fetchFuture = fetchExecutor.submit(() -> { - fetch(); - return null; - }); - fetchFutures.add(fetchFuture); - } - } - - /** - * Fetch objects to a local disk up to {@link FetchConfig#maxFetchCapacityBytes}. - * This method is not thread safe and must be called by a single thread. Note that even - * {@link FetchConfig#maxFetchCapacityBytes} is 0, at least 1 file is always fetched. - * This is for simplifying design, and should be improved when our client implementations for cloud storages - * like S3 support range scan. - *

- * This method is called by {@link #fetchExecutor} if prefetch is enabled. Otherwise, it is called by the same - * thread. - */ - private void fetch() throws Exception - { - for (; nextFetchIndex < objects.size() - && fetchedBytes.get() <= fetchConfig.getMaxFetchCapacityBytes(); nextFetchIndex++) { - final T object = objects.get(nextFetchIndex); - LOG.info("Fetching [%d]th object[%s], fetchedBytes[%d]", nextFetchIndex, object, fetchedBytes.get()); - final File outFile = File.createTempFile(FETCH_FILE_PREFIX, null, temporaryDirectory); - fetchedBytes.addAndGet(download(object, outFile)); - fetchedFiles.put(new FetchedFile<>(object, outFile, getFileCloser(outFile, fetchedBytes))); - } - } - - /** - * Downloads an object into a file. The download process could be retried depending on the object source. - * - * @param object an object to be downloaded - * @param outFile a file which the object data is stored - * - * @return number of downloaded bytes - */ - protected abstract long download(T object, File outFile) throws IOException; - - /** - * Generates an instance of {@link OpenObject} for the given object. - */ - protected abstract OpenObject generateOpenObject(T object) throws IOException; - - protected FetchConfig getFetchConfig() - { - return fetchConfig; - } - - @Override - public boolean hasNext() - { - return numRemainingObjects > 0; - } - - @Override - public OpenObject next() - { - if (!hasNext()) { - throw new NoSuchElementException(); - } - - // If fetch() fails, hasNext() always returns true and next() is always called. The below method checks that - // fetch() threw an exception and propagates it if exists. - checkFetchException(false); - - try { - final OpenObject openObject = prefetchEnabled ? openObjectFromLocal() : openObjectFromRemote(); - numRemainingObjects--; - return openObject; - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - private void checkFetchException(boolean wait) - { - try { - for (Future fetchFuture; (fetchFuture = fetchFutures.poll()) != null; ) { - if (wait) { - fetchFuture.get(fetchConfig.getFetchTimeout(), TimeUnit.MILLISECONDS); - } else { - if (fetchFuture.isDone()) { - fetchFuture.get(); - } else { - fetchFutures.addFirst(fetchFuture); - break; - } - } - } - } - catch (InterruptedException | ExecutionException e) { - throw new RuntimeException(e); - } - catch (TimeoutException e) { - throw new ISE(e, "Failed to fetch, but cannot check the reason in [%d] ms", fetchConfig.getFetchTimeout()); - } - } - - private OpenObject openObjectFromLocal() throws IOException - { - final FetchedFile fetchedFile; - - if (!fetchedFiles.isEmpty()) { - // If there are already fetched files, use them - fetchedFile = fetchedFiles.poll(); - } else { - // Otherwise, wait for fetching - try { - fetchIfNeeded(fetchedBytes.get()); - fetchedFile = fetchedFiles.poll(fetchConfig.getFetchTimeout(), TimeUnit.MILLISECONDS); - if (fetchedFile == null) { - // Check the latest fetch is failed - checkFetchException(true); - // Or throw a timeout exception - throw new RuntimeException(new TimeoutException()); - } - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - final FetchedFile maybeCached = cacheIfPossible(fetchedFile); - // trigger fetch again for subsequent next() calls - fetchIfNeeded(fetchedBytes.get()); - return new OpenObject<>(maybeCached); - } - - private OpenObject openObjectFromRemote() throws IOException - { - if (fetchedFiles.size() > 0) { - // If fetchedFiles is not empty even though prefetching is disabled, they should be cached files. - // We use them first. See (*). - return new OpenObject<>(fetchedFiles.poll()); - } else if (cacheManager.cacheable()) { - // If cache is enabled, first download an object to local storage and cache it. - try { - // Since maxFetchCapacityBytes is 0, at most one file is fetched. - fetch(); - FetchedFile fetchedFile = fetchedFiles.poll(); - if (fetchedFile == null) { - throw new ISE("Cannot fetch object[%s]", objects.get(nextFetchIndex - 1)); - } - final FetchedFile cached = cacheIfPossible(fetchedFile); - return new OpenObject<>(cached); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } else { - final T object = objects.get(nextFetchIndex); - LOG.info("Reading [%d]th object[%s]", nextFetchIndex, object); - nextFetchIndex++; - return generateOpenObject(object); - } - } - - private FetchedFile cacheIfPossible(FetchedFile fetchedFile) - { - if (cacheManager.cacheable()) { - final FetchedFile cachedFile = cacheManager.cache(fetchedFile); - // If the fetchedFile is cached, make a room for fetching more data immediately. - // This is because cache space and fetch space are separated. - fetchedBytes.addAndGet(-fetchedFile.length()); - return cachedFile; - } else { - return fetchedFile; - } - } - - private static Closeable getFileCloser( - final File file, - final AtomicLong fetchedBytes - ) - { - return () -> { - final long fileSize = file.length(); - file.delete(); - fetchedBytes.addAndGet(-fileSize); - }; - } -} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/FileFetcher.java b/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/FileFetcher.java deleted file mode 100644 index 7d869dacf71..00000000000 --- a/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/FileFetcher.java +++ /dev/null @@ -1,111 +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.druid.data.input.impl.prefetch; - -import com.google.common.base.Predicate; -import org.apache.druid.data.input.impl.RetryingInputStream; -import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.StringUtils; - -import javax.annotation.Nullable; -import java.io.Closeable; -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.concurrent.ExecutorService; - -/** - * A file fetcher used by {@link PrefetchableTextFilesFirehoseFactory}. - * See the javadoc of {@link PrefetchableTextFilesFirehoseFactory} for more details. - */ -public class FileFetcher extends Fetcher -{ - private static final int BUFFER_SIZE = 1024 * 4; - private final ObjectOpenFunction openObjectFunction; - private final Predicate retryCondition; - private final byte[] buffer; - - FileFetcher( - CacheManager cacheManager, - List objects, - ExecutorService fetchExecutor, - @Nullable File temporaryDirectory, - FetchConfig fetchConfig, - ObjectOpenFunction openObjectFunction, - Predicate retryCondition - ) - { - - super( - cacheManager, - objects, - fetchExecutor, - temporaryDirectory, - fetchConfig - ); - - this.openObjectFunction = openObjectFunction; - this.retryCondition = retryCondition; - this.buffer = new byte[BUFFER_SIZE]; - } - - /** - * Downloads an object. It retries downloading {@link FetchConfig#maxFetchRetry} - * times and throws an exception. - * - * @param object an object to be downloaded - * @param outFile a file which the object data is stored - * - * @return number of downloaded bytes - */ - @Override - protected long download(T object, File outFile) throws IOException - { - return FileUtils.copyLarge( - object, - openObjectFunction, - outFile, - buffer, - retryCondition, - getFetchConfig().getMaxFetchRetry() + 1, - StringUtils.format("Failed to download object[%s]", object) - ); - } - - /** - * Generates an instance of {@link OpenObject} for which the underlying stream may be re-opened and retried - * based on the exception and retry condition. - */ - @Override - protected OpenObject generateOpenObject(T object) throws IOException - { - return new OpenObject<>( - object, - new RetryingInputStream<>(object, openObjectFunction, retryCondition, getFetchConfig().getMaxFetchRetry()), - getNoopCloser() - ); - } - - private static Closeable getNoopCloser() - { - return () -> { - }; - } -} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/ObjectOpenFunction.java b/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/ObjectOpenFunction.java index 6af64fb18cf..6941e82df98 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/ObjectOpenFunction.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/ObjectOpenFunction.java @@ -19,7 +19,6 @@ package org.apache.druid.data.input.impl.prefetch; -import java.io.File; import java.io.IOException; import java.io.InputStream; @@ -31,10 +30,4 @@ public interface ObjectOpenFunction { return open(object); } - - default InputStream open(T object, File outFile) throws IOException - { - return open(object); - } - } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/OpenObject.java b/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/OpenObject.java deleted file mode 100644 index 07b3125099c..00000000000 --- a/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/OpenObject.java +++ /dev/null @@ -1,71 +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.druid.data.input.impl.prefetch; - -import org.apache.commons.io.FileUtils; - -import java.io.Closeable; -import java.io.IOException; -import java.io.InputStream; - -/** - * A class containing meta information about an opened object. This class is used to put related objects together. It - * contains an original object, an objectStream from the object, and a resourceCloser which knows how to release - * associated resources on closing. - * - * {@link PrefetchableTextFilesFirehoseFactory.ResourceCloseableLineIterator} consumes the objectStream and closes - * it with the resourceCloser. - */ -public class OpenObject -{ - // Original object - private final T object; - // Input stream from the object - private final InputStream objectStream; - // Closer which is called when the file is not needed anymore. Usually this deletes the file except for cached files. - private final Closeable resourceCloser; - - public OpenObject(FetchedFile fetchedFile) throws IOException - { - this(fetchedFile.getObject(), FileUtils.openInputStream(fetchedFile.getFile()), fetchedFile.getResourceCloser()); - } - - public OpenObject(T object, InputStream objectStream, Closeable resourceCloser) - { - this.object = object; - this.objectStream = objectStream; - this.resourceCloser = resourceCloser; - } - - public T getObject() - { - return object; - } - - public InputStream getObjectStream() - { - return objectStream; - } - - public Closeable getResourceCloser() - { - return resourceCloser; - } -} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactory.java b/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactory.java deleted file mode 100644 index 50eefae7366..00000000000 --- a/processing/src/main/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactory.java +++ /dev/null @@ -1,318 +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.druid.data.input.impl.prefetch; - -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.collect.ImmutableList; -import org.apache.commons.io.LineIterator; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.impl.AbstractTextFilesFirehoseFactory; -import org.apache.druid.data.input.impl.FileIteratingFirehose; -import org.apache.druid.data.input.impl.RetryingInputStream; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.logger.Logger; - -import javax.annotation.Nullable; -import java.io.Closeable; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.io.Reader; -import java.nio.charset.StandardCharsets; -import java.util.Iterator; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; - -/** - * PrefetchableTextFilesFirehoseFactory is an abstract firehose factory for reading text files. The firehose returned - * by this class provides three key functionalities. - *

- *

- * - Caching: for the first call of {@link #connect(StringInputRowParser, File)}, it caches objects in a local disk - * up to maxCacheCapacityBytes. These caches are NOT deleted until the process terminates, and thus can be used for - * future reads. - *
- * - Fetching: when it reads all cached data, it fetches remaining objects into a local disk and reads data from - * them. For the performance reason, prefetch technique is used, that is, when the size of remaining fetched data is - * smaller than {@link FetchConfig#prefetchTriggerBytes}, a background prefetch thread automatically starts to fetch remaining - * objects. - *
- * - Retry: if an exception occurs while downloading an object, it retries again up to {@link FetchConfig#maxFetchRetry}. - *

- *

- * This implementation can be useful when the cost for reading input objects is large as reading from AWS S3 because - * batch tasks like IndexTask or HadoopIndexTask can read the whole data twice for determining partition specs and - * generating segments if the intervals of GranularitySpec is not specified. - *
- * Prefetching can be turned on/off by setting maxFetchCapacityBytes. Depending on prefetching is enabled or - * disabled, the behavior of the firehose is different like below. - *

- *

- * 1. If prefetch is enabled, this firehose can fetch input objects in background. - *
- * 2. When next() is called, it first checks that there are already fetched files in local storage. - *
- * 2.1 If exists, it simply chooses a fetched file and returns a {@link LineIterator} reading that file. - *
- * 2.2 If there is no fetched files in local storage but some objects are still remained to be read, the firehose - * fetches one of input objects in background immediately. If an IOException occurs while downloading the object, - * it retries up to the maximum retry count. Finally, the firehose returns a {@link LineIterator} only when the - * download operation is successfully finished. - *
- * 3. If prefetch is disabled, the firehose returns a {@link LineIterator} which directly reads the stream opened by - * {@link #openObjectStream}. If there is an IOException, it will throw it and the read will fail. - */ -public abstract class PrefetchableTextFilesFirehoseFactory - extends AbstractTextFilesFirehoseFactory -{ - private static final Logger LOG = new Logger(PrefetchableTextFilesFirehoseFactory.class); - - private static final CacheManager DISABLED_CACHE_MANAGER = new CacheManager(0); - private static final FetchConfig DISABLED_PREFETCH_CONFIG = new FetchConfig(0L, 0L, 0L, 0L, 0); - - private final CacheManager cacheManager; - private final FetchConfig fetchConfig; - - private List objects; - - public PrefetchableTextFilesFirehoseFactory( - @Nullable Long maxCacheCapacityBytes, - @Nullable Long maxFetchCapacityBytes, - @Nullable Long prefetchTriggerBytes, - @Nullable Long fetchTimeout, - @Nullable Integer maxFetchRetry - ) - { - this.fetchConfig = new FetchConfig( - maxCacheCapacityBytes, - maxFetchCapacityBytes, - prefetchTriggerBytes, - fetchTimeout, - maxFetchRetry - ); - this.cacheManager = new CacheManager<>( - fetchConfig.getMaxCacheCapacityBytes() - ); - } - - @JsonProperty - public long getMaxCacheCapacityBytes() - { - return cacheManager.getMaxCacheCapacityBytes(); - } - - @JsonProperty - public long getMaxFetchCapacityBytes() - { - return fetchConfig.getMaxFetchCapacityBytes(); - } - - @JsonProperty - public long getPrefetchTriggerBytes() - { - return fetchConfig.getPrefetchTriggerBytes(); - } - - @JsonProperty - public long getFetchTimeout() - { - return fetchConfig.getFetchTimeout(); - } - - @JsonProperty - public int getMaxFetchRetry() - { - return fetchConfig.getMaxFetchRetry(); - } - - @VisibleForTesting - CacheManager getCacheManager() - { - return cacheManager; - } - - @Override - public Firehose connect(StringInputRowParser firehoseParser, @Nullable File temporaryDirectory) throws IOException - { - return connectInternal(firehoseParser, temporaryDirectory, this.fetchConfig, this.cacheManager); - } - - @Override - public Firehose connectForSampler(StringInputRowParser parser, @Nullable File temporaryDirectory) throws IOException - { - return connectInternal(parser, temporaryDirectory, DISABLED_PREFETCH_CONFIG, DISABLED_CACHE_MANAGER); - } - - private Firehose connectInternal( - StringInputRowParser firehoseParser, - @Nullable File temporaryDirectory, - FetchConfig fetchConfig, - CacheManager cacheManager - ) throws IOException - { - if (objects == null) { - objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "objects")); - } - - if (cacheManager.isEnabled() || fetchConfig.getMaxFetchCapacityBytes() > 0) { - Preconditions.checkNotNull(temporaryDirectory, "temporaryDirectory"); - Preconditions.checkArgument( - temporaryDirectory.exists(), - "temporaryDirectory[%s] does not exist", - temporaryDirectory - ); - Preconditions.checkArgument( - temporaryDirectory.isDirectory(), - "temporaryDirectory[%s] is not a directory", - temporaryDirectory - ); - } - - LOG.info("Create a new firehose for [%d] objects", objects.size()); - - // fetchExecutor is responsible for background data fetching - final ExecutorService fetchExecutor = Execs.singleThreaded("firehose_fetch_%d"); - final FileFetcher fetcher = new FileFetcher( - cacheManager, - objects, - fetchExecutor, - temporaryDirectory, - fetchConfig, - new ObjectOpenFunction() - { - @Override - public InputStream open(T object) throws IOException - { - return openObjectStream(object); - } - - @Override - public InputStream open(T object, long start) throws IOException - { - return openObjectStream(object, start); - } - }, - getRetryCondition() - ); - - return new FileIteratingFirehose( - new Iterator() - { - @Override - public boolean hasNext() - { - return fetcher.hasNext(); - } - - @Override - public LineIterator next() - { - if (!hasNext()) { - throw new NoSuchElementException(); - } - - final OpenObject openObject = fetcher.next(); - try { - return new ResourceCloseableLineIterator( - new InputStreamReader( - wrapObjectStream(openObject.getObject(), openObject.getObjectStream()), - StandardCharsets.UTF_8 - ), - openObject.getResourceCloser() - ); - } - catch (IOException e) { - try { - openObject.getResourceCloser().close(); - } - catch (Throwable t) { - e.addSuppressed(t); - } - throw new RuntimeException(e); - } - } - }, - firehoseParser, - () -> { - fetchExecutor.shutdownNow(); - try { - Preconditions.checkState(fetchExecutor.awaitTermination( - fetchConfig.getFetchTimeout(), - TimeUnit.MILLISECONDS - )); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new ISE("Failed to shutdown fetch executor during close"); - } - } - ); - } - - /** - * Returns a predicate describing retry conditions. {@link Fetcher} and {@link RetryingInputStream} will retry on the - * errors satisfying this condition. - */ - protected abstract Predicate getRetryCondition(); - - /** - * Open an input stream from the given object. If the object is compressed, this method should return a byte stream - * as it is compressed. The object compression should be handled in {@link #wrapObjectStream(Object, InputStream)}. - * - * @param object an object to be read - * @param start start offset - * - * @return an input stream for the object - */ - protected abstract InputStream openObjectStream(T object, long start) throws IOException; - - /** - * This class calls the {@link Closeable#close()} method of the resourceCloser when it is closed. - */ - static class ResourceCloseableLineIterator extends LineIterator - { - private final Closeable resourceCloser; - - ResourceCloseableLineIterator(Reader reader, Closeable resourceCloser) throws IllegalArgumentException - { - super(reader); - this.resourceCloser = resourceCloser; - } - - @Override - public void close() - { - try (Closeable ignore = this.resourceCloser) { - super.close(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - } -} diff --git a/processing/src/main/java/org/apache/druid/segment/transform/TransformSpec.java b/processing/src/main/java/org/apache/druid/segment/transform/TransformSpec.java index 1391da394b0..62e85f2c84f 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/TransformSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/TransformSpec.java @@ -66,21 +66,6 @@ public class TransformSpec } } - public static TransformSpec fromInputRowParser(final InputRowParser parser) - { - // Hack: some firehoses and input specs must extract transformSpec from the parser, since they do not - // actually use the parser, but still must respect the transformSpec. This method should extract whatever - // transformSpec "decorate" had put in. - - if (parser instanceof TransformingInputRowParser) { - return ((TransformingInputRowParser) parser).getTransformSpec(); - } else if (parser instanceof TransformingStringInputRowParser) { - return ((TransformingStringInputRowParser) parser).getTransformSpec(); - } else { - throw new ISE("Parser was not decorated, but should have been"); - } - } - @JsonProperty @Nullable public DimFilter getFilter() diff --git a/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputRowParser.java b/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputRowParser.java index ea6da0b36a6..53204314f4d 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputRowParser.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputRowParser.java @@ -57,9 +57,4 @@ public class TransformingInputRowParser implements InputRowParser { return new TransformingInputRowParser<>(parser.withParseSpec(parseSpec), transformSpec); } - - public TransformSpec getTransformSpec() - { - return transformSpec; - } } diff --git a/processing/src/main/java/org/apache/druid/segment/transform/TransformingStringInputRowParser.java b/processing/src/main/java/org/apache/druid/segment/transform/TransformingStringInputRowParser.java index d9a0aa2d954..cba455d4691 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/TransformingStringInputRowParser.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/TransformingStringInputRowParser.java @@ -62,9 +62,4 @@ public class TransformingStringInputRowParser extends StringInputRowParser { return new TransformingStringInputRowParser(parseSpec, getEncoding(), transformSpec); } - - public TransformSpec getTransformSpec() - { - return transformSpec; - } } diff --git a/processing/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java b/processing/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java deleted file mode 100644 index 3def2829327..00000000000 --- a/processing/src/test/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptorTest.java +++ /dev/null @@ -1,158 +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.druid.data.input; - -import com.google.common.collect.Iterables; -import org.apache.druid.data.input.impl.CSVParseSpec; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.java.util.common.parsers.ParseException; -import org.apache.druid.testing.InitializedNullHandlingTest; -import org.junit.Assert; -import org.junit.Test; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.stream.Stream; - -public class FirehoseFactoryToInputSourceAdaptorTest extends InitializedNullHandlingTest -{ - @Test - public void testUnimplementedInputFormat() throws IOException - { - final List lines = new ArrayList<>(); - for (int i = 0; i < 10; i++) { - lines.add(StringUtils.format("%d,name_%d,%d", 20190101 + i, i, i + 100)); - } - final TestFirehoseFactory firehoseFactory = new TestFirehoseFactory(lines); - final StringInputRowParser inputRowParser = new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec(null, "yyyyMMdd", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "name", "score"))), - ",", - Arrays.asList("timestamp", "name", "score"), - false, - 0 - ), - StringUtils.UTF8_STRING - ); - final FirehoseFactoryToInputSourceAdaptor inputSourceAdaptor = new FirehoseFactoryToInputSourceAdaptor( - firehoseFactory, - inputRowParser - ); - final InputSourceReader reader = inputSourceAdaptor.reader( - new InputRowSchema( - inputRowParser.getParseSpec().getTimestampSpec(), - inputRowParser.getParseSpec().getDimensionsSpec(), - ColumnsFilter.all() - ), - null, - null - ); - final List result = new ArrayList<>(); - try (CloseableIterator iterator = reader.read()) { - while (iterator.hasNext()) { - result.add(iterator.next()); - } - } - Assert.assertEquals(10, result.size()); - for (int i = 0; i < 10; i++) { - Assert.assertEquals(DateTimes.of(StringUtils.format("2019-01-%02d", 1 + i)), result.get(i).getTimestamp()); - Assert.assertEquals( - StringUtils.format("name_%d", i), - Iterables.getOnlyElement(result.get(i).getDimension("name")) - ); - Assert.assertEquals( - StringUtils.format("%d", i + 100), - Iterables.getOnlyElement(result.get(i).getDimension("score")) - ); - } - } - - private static class TestFirehoseFactory implements FiniteFirehoseFactory - { - private final List lines; - - private TestFirehoseFactory(List lines) - { - this.lines = lines; - } - - @Override - public Firehose connect(StringInputRowParser parser, @Nullable File temporaryDirectory) throws ParseException - { - return new Firehose() - { - final Iterator iterator = lines.iterator(); - - @Override - public boolean hasMore() - { - return iterator.hasNext(); - } - - @Override - public InputRow nextRow() - { - return parser.parse(iterator.next()); - } - - @Override - public void close() - { - // do nothing - } - }; - } - - @Override - public boolean isSplittable() - { - return false; - } - - @Override - public Stream> getSplits(@Nullable SplitHintSpec splitHintSpec) - { - return null; - } - - @Override - public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) - { - return 0; - } - - @Override - public FiniteFirehoseFactory withSplit(InputSplit split) - { - return null; - } - } -} diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/FileIteratingFirehoseTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/FileIteratingFirehoseTest.java deleted file mode 100644 index 14c2382c89a..00000000000 --- a/processing/src/test/java/org/apache/druid/data/input/impl/FileIteratingFirehoseTest.java +++ /dev/null @@ -1,171 +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.druid.data.input.impl; - -import com.google.common.base.Joiner; -import com.google.common.collect.ImmutableList; -import org.apache.commons.io.LineIterator; -import org.apache.druid.common.config.NullHandlingTest; -import org.apache.druid.data.input.InputRow; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -import java.io.Closeable; -import java.io.IOException; -import java.io.Reader; -import java.io.StringReader; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -@RunWith(Parameterized.class) -public class FileIteratingFirehoseTest extends NullHandlingTest -{ - @Parameters(name = "{0}, {1}") - public static Collection constructorFeeder() - { - final List> inputTexts = ImmutableList.of( - ImmutableList.of("2000,foo"), - ImmutableList.of("2000,foo\n2000,bar\n"), - ImmutableList.of("2000,foo\n2000,bar\n", "2000,baz"), - ImmutableList.of("2000,foo\n2000,bar\n", "", "2000,baz"), - ImmutableList.of("2000,foo\n2000,bar\n", "", "2000,baz", ""), - ImmutableList.of("2000,foo\n2000,bar\n2000,baz", "", "2000,baz", "2000,foo\n2000,bar\n3000,baz"), - ImmutableList.of(""), - ImmutableList.of() - ); - - final List args = new ArrayList<>(); - for (int numSkipHeadRows = 0; numSkipHeadRows < 3; numSkipHeadRows++) { - for (List texts : inputTexts) { - args.add(new Object[] {texts, numSkipHeadRows}); - } - } - - return args; - } - - private static final char[] LINE_CHARS = "\n".toCharArray(); - - private final StringInputRowParser parser; - private final List inputs; - private final List expectedResults; - - public FileIteratingFirehoseTest(List texts, int numSkipHeaderRows) - { - parser = new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec("ts", "auto", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("x"))), - ",", - ImmutableList.of("ts", "x"), - false, - numSkipHeaderRows - ), - null - ); - - this.inputs = texts; - this.expectedResults = inputs.stream() - .map(input -> input.split("\n")) - .flatMap(lines -> { - final List filteredLines = Arrays - .stream(lines) - .filter(line -> line.length() > 0) - .map(line -> line.split(",")[1]) - .collect(Collectors.toList()); - - final int numRealSkippedRows = Math.min(filteredLines.size(), numSkipHeaderRows); - IntStream.range(0, numRealSkippedRows).forEach(i -> filteredLines.set(i, null)); - return filteredLines.stream(); - }) - .collect(Collectors.toList()); - } - - @Test - public void testFirehose() throws Exception - { - final List lineIterators = inputs.stream() - .map(s -> new LineIterator(new StringReader(s))) - .collect(Collectors.toList()); - - try (final FileIteratingFirehose firehose = new FileIteratingFirehose(lineIterators.iterator(), parser)) { - final List results = new ArrayList<>(); - - while (firehose.hasMore()) { - final InputRow inputRow = firehose.nextRow(); - if (inputRow == null) { - results.add(null); - } else { - results.add(Joiner.on("|").join(inputRow.getDimension("x"))); - } - } - - Assert.assertEquals(expectedResults, results); - } - } - - @Test(expected = RuntimeException.class) - public void testClose() throws IOException - { - final LineIterator lineIterator = new LineIterator(new Reader() - { - @Override - public int read(char[] cbuf, int off, int len) - { - System.arraycopy(LINE_CHARS, 0, cbuf, 0, LINE_CHARS.length); - return LINE_CHARS.length; - } - - @Override - public void close() - { - throw new RuntimeException("close test for FileIteratingFirehose"); - } - }); - - final TestCloseable closeable = new TestCloseable(); - final FileIteratingFirehose firehose = new FileIteratingFirehose( - ImmutableList.of(lineIterator).iterator(), - parser, - closeable - ); - firehose.hasMore(); // initialize lineIterator - firehose.close(); - Assert.assertTrue(closeable.closed); - } - - private static final class TestCloseable implements Closeable - { - private boolean closed; - - @Override - public void close() - { - closed = true; - } - } -} diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java b/processing/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java deleted file mode 100644 index 2669a968d04..00000000000 --- a/processing/src/test/java/org/apache/druid/data/input/impl/NoopFirehoseFactory.java +++ /dev/null @@ -1,54 +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.druid.data.input.impl; - -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.SplitHintSpec; - -import javax.annotation.Nullable; -import java.util.stream.Stream; - -public class NoopFirehoseFactory implements FiniteFirehoseFactory -{ - @Override - public String toString() - { - return "NoopFirehoseFactory{}"; - } - - @Override - public Stream getSplits(@Nullable SplitHintSpec splitHintSpec) - { - throw new UnsupportedOperationException(); - } - - @Override - public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) - { - throw new UnsupportedOperationException(); - } - - @Override - public FiniteFirehoseFactory withSplit(InputSplit split) - { - throw new UnsupportedOperationException(); - } -} diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java deleted file mode 100644 index 38fb843a34c..00000000000 --- a/processing/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java +++ /dev/null @@ -1,643 +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.druid.data.input.impl.prefetch; - -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.io.CountingOutputStream; -import io.netty.util.SuppressForbidden; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.filefilter.TrueFileFilter; -import org.apache.druid.common.config.NullHandling; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.Row; -import org.apache.druid.data.input.impl.CSVParseSpec; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.RetryUtils; -import org.apache.druid.java.util.common.StringUtils; -import org.hamcrest.CoreMatchers; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Ignore; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.rules.TemporaryFolder; - -import java.io.BufferedWriter; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStreamWriter; -import java.io.Writer; -import java.net.SocketException; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeoutException; - -public class PrefetchableTextFilesFirehoseFactoryTest -{ - private static long FILE_SIZE = -1; - - private static final StringInputRowParser PARSER = new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec( - "timestamp", - "auto", - null - ), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a", "b"))), - ",", - Arrays.asList("timestamp", "a", "b"), - false, - 0 - ), - StandardCharsets.UTF_8.name() - ); - - @ClassRule - public static TemporaryFolder tempDir = new TemporaryFolder(); - private static File TEST_DIR; - - @Rule - public ExpectedException expectedException = ExpectedException.none(); - - @BeforeClass - public static void setup() throws IOException - { - NullHandling.initializeForTests(); - TEST_DIR = tempDir.newFolder(); - for (int i = 0; i < 100; i++) { - try ( - CountingOutputStream cos = new CountingOutputStream( - Files.newOutputStream(new File(TEST_DIR, "test_" + i).toPath()) - ); - Writer writer = new BufferedWriter(new OutputStreamWriter(cos, StandardCharsets.UTF_8)) - ) { - for (int j = 0; j < 100; j++) { - final String a = StringUtils.format("%d,%03d,%03d\n", (20171220 + i), i, j); - writer.write(a); - } - writer.flush(); - // Every file size must be same - if (FILE_SIZE == -1) { - FILE_SIZE = cos.getCount(); - } else { - Assert.assertEquals(FILE_SIZE, cos.getCount()); - } - } - } - } - - private static void assertResult(List rows) - { - Assert.assertEquals(10000, rows.size()); - rows.sort((r1, r2) -> { - int c = r1.getTimestamp().compareTo(r2.getTimestamp()); - if (c != 0) { - return c; - } - c = Integer.valueOf(r1.getDimension("a").get(0)).compareTo(Integer.valueOf(r2.getDimension("a").get(0))); - if (c != 0) { - return c; - } - - return Integer.valueOf(r1.getDimension("b").get(0)).compareTo(Integer.valueOf(r2.getDimension("b").get(0))); - }); - - for (int i = 0; i < 100; i++) { - for (int j = 0; j < 100; j++) { - final Row row = rows.get(i * 100 + j); - Assert.assertEquals(DateTimes.utc(20171220 + i), row.getTimestamp()); - Assert.assertEquals(i, Integer.valueOf(row.getDimension("a").get(0)).intValue()); - Assert.assertEquals(j, Integer.valueOf(row.getDimension("b").get(0)).intValue()); - } - } - } - - private static void assertNumRemainingCacheFiles(File firehoseTmpDir, int expectedNumFiles) - { - final String[] files = firehoseTmpDir.list(); - Assert.assertNotNull(files); - Assert.assertEquals(expectedNumFiles, files.length); - } - - @SuppressForbidden(reason = "Files#createTempDirectory") - private static File createFirehoseTmpDir(String dirPrefix) throws IOException - { - return Files.createTempDirectory(tempDir.getRoot().toPath(), dirPrefix).toFile(); - } - - @Test - public void testWithoutCacheAndFetch() throws IOException - { - final TestPrefetchableTextFilesFirehoseFactory factory = - TestPrefetchableTextFilesFirehoseFactory.with(TEST_DIR, 0, 0); - - final List rows = new ArrayList<>(); - final File firehoseTmpDir = createFirehoseTmpDir("testWithoutCacheAndFetch"); - try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - - Assert.assertEquals(0, factory.getCacheManager().getTotalCachedBytes()); - assertResult(rows); - assertNumRemainingCacheFiles(firehoseTmpDir, 0); - } - - @Test - public void testWithoutCacheAndFetchAgainstConnectionReset() throws IOException - { - final TestPrefetchableTextFilesFirehoseFactory factory = - TestPrefetchableTextFilesFirehoseFactory.withConnectionResets(TEST_DIR, 0, 0, 2); - - final List rows = new ArrayList<>(); - final File firehoseTmpDir = createFirehoseTmpDir("testWithoutCacheAndFetch"); - try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - - Assert.assertEquals(0, factory.getCacheManager().getTotalCachedBytes()); - assertResult(rows); - assertNumRemainingCacheFiles(firehoseTmpDir, 0); - } - - @Test - public void testWithoutCache() throws IOException - { - final TestPrefetchableTextFilesFirehoseFactory factory = - TestPrefetchableTextFilesFirehoseFactory.with(TEST_DIR, 0, 2048); - - final List rows = new ArrayList<>(); - final File firehoseTmpDir = createFirehoseTmpDir("testWithoutCache"); - try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - - Assert.assertEquals(0, factory.getCacheManager().getTotalCachedBytes()); - assertResult(rows); - assertNumRemainingCacheFiles(firehoseTmpDir, 0); - } - - @Test - public void testWithZeroFetchCapacity() throws IOException - { - final TestPrefetchableTextFilesFirehoseFactory factory = - TestPrefetchableTextFilesFirehoseFactory.with(TEST_DIR, 2048, 0); - - final List rows = new ArrayList<>(); - final File firehoseTmpDir = createFirehoseTmpDir("testWithZeroFetchCapacity"); - try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - - assertResult(rows); - assertNumRemainingCacheFiles(firehoseTmpDir, 2); - } - - @Test - public void testWithCacheAndFetch() throws IOException - { - final TestPrefetchableTextFilesFirehoseFactory factory = - TestPrefetchableTextFilesFirehoseFactory.of(TEST_DIR); - - final List rows = new ArrayList<>(); - final File firehoseTmpDir = createFirehoseTmpDir("testWithCacheAndFetch"); - try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - - assertResult(rows); - assertNumRemainingCacheFiles(firehoseTmpDir, 2); - } - - @Test - @Ignore("See issue #12638") - public void testWithLargeCacheAndSmallFetch() throws IOException - { - final TestPrefetchableTextFilesFirehoseFactory factory = - TestPrefetchableTextFilesFirehoseFactory.with(TEST_DIR, 2048, 1024); - - final List rows = new ArrayList<>(); - final File firehoseTmpDir = createFirehoseTmpDir("testWithLargeCacheAndSmallFetch"); - try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - - assertResult(rows); - assertNumRemainingCacheFiles(firehoseTmpDir, 2); - } - - @Test - public void testWithSmallCacheAndLargeFetch() throws IOException - { - final TestPrefetchableTextFilesFirehoseFactory factory = - TestPrefetchableTextFilesFirehoseFactory.with(TEST_DIR, 1024, 2048); - - final List rows = new ArrayList<>(); - final File firehoseTmpDir = createFirehoseTmpDir("testWithSmallCacheAndLargeFetch"); - try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - - assertResult(rows); - assertNumRemainingCacheFiles(firehoseTmpDir, 1); - } - - @Test - public void testRetry() throws IOException - { - final TestPrefetchableTextFilesFirehoseFactory factory = - TestPrefetchableTextFilesFirehoseFactory.withOpenExceptions(TEST_DIR, 1); - - final List rows = new ArrayList<>(); - final File firehoseTmpDir = createFirehoseTmpDir("testRetry"); - try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - - assertResult(rows); - assertNumRemainingCacheFiles(firehoseTmpDir, 2); - } - - @Test - public void testMaxRetry() throws IOException - { - expectedException.expect(RuntimeException.class); - expectedException.expectCause(CoreMatchers.instanceOf(ExecutionException.class)); - expectedException.expectMessage("Exception for retry test"); - - final TestPrefetchableTextFilesFirehoseFactory factory = - TestPrefetchableTextFilesFirehoseFactory.withOpenExceptions(TEST_DIR, 5); - - try (Firehose firehose = factory.connect(PARSER, createFirehoseTmpDir("testMaxRetry"))) { - while (firehose.hasMore()) { - firehose.nextRow(); - } - } - } - - @Test - public void testTimeout() throws IOException - { - expectedException.expect(RuntimeException.class); - expectedException.expectCause(CoreMatchers.instanceOf(TimeoutException.class)); - - final TestPrefetchableTextFilesFirehoseFactory factory = - TestPrefetchableTextFilesFirehoseFactory.withSleepMillis(TEST_DIR, 1000); - - try (Firehose firehose = factory.connect(PARSER, createFirehoseTmpDir("testTimeout"))) { - while (firehose.hasMore()) { - firehose.nextRow(); - } - } - } - - @Test - @Ignore("See issue #12638") - public void testReconnectWithCacheAndPrefetch() throws IOException - { - final TestPrefetchableTextFilesFirehoseFactory factory = - TestPrefetchableTextFilesFirehoseFactory.of(TEST_DIR); - final File firehoseTmpDir = createFirehoseTmpDir("testReconnectWithCacheAndPrefetch"); - - for (int i = 0; i < 5; i++) { - final List rows = new ArrayList<>(); - try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) { - if (i > 0) { - Assert.assertEquals(FILE_SIZE * 2, factory.getCacheManager().getTotalCachedBytes()); - } - - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - assertResult(rows); - assertNumRemainingCacheFiles(firehoseTmpDir, 2); - } - } - - @Test - public void testReconnectWithCache() throws IOException - { - final TestPrefetchableTextFilesFirehoseFactory factory = - TestPrefetchableTextFilesFirehoseFactory.with(TEST_DIR, 2048, 0); - final File firehoseTmpDir = createFirehoseTmpDir("testReconnectWithCache"); - - for (int i = 0; i < 5; i++) { - final List rows = new ArrayList<>(); - try (Firehose firehose = factory.connect(PARSER, firehoseTmpDir)) { - if (i > 0) { - Assert.assertEquals(FILE_SIZE * 2, factory.getCacheManager().getTotalCachedBytes()); - } - - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - assertResult(rows); - assertNumRemainingCacheFiles(firehoseTmpDir, 2); - } - } - - static class TestPrefetchableTextFilesFirehoseFactory extends PrefetchableTextFilesFirehoseFactory - { - private final long sleepMillis; - private final File baseDir; - private int numOpenExceptions; - private int maxConnectionResets; - - static TestPrefetchableTextFilesFirehoseFactory with(File baseDir, long cacheCapacity, long fetchCapacity) - { - return new TestPrefetchableTextFilesFirehoseFactory( - baseDir, - 1024, - cacheCapacity, - fetchCapacity, - 60_000, // fetch timeout - 3, - 0, - 0, - 0 - ); - } - - static TestPrefetchableTextFilesFirehoseFactory of(File baseDir) - { - return new TestPrefetchableTextFilesFirehoseFactory( - baseDir, - 1024, - 2048, - 2048, - 3, - 0, - 0, - 0 - ); - } - - static TestPrefetchableTextFilesFirehoseFactory withOpenExceptions(File baseDir, int count) - { - return new TestPrefetchableTextFilesFirehoseFactory( - baseDir, - 1024, - 2048, - 2048, - 3, - count, - 0, - 0 - ); - } - - static TestPrefetchableTextFilesFirehoseFactory withConnectionResets( - File baseDir, - long cacheCapacity, - long fetchCapacity, - int numConnectionResets - ) - { - return new TestPrefetchableTextFilesFirehoseFactory( - baseDir, - fetchCapacity / 2, - cacheCapacity, - fetchCapacity, - 3, - 0, - numConnectionResets, - 0 - ); - } - - static TestPrefetchableTextFilesFirehoseFactory withSleepMillis(File baseDir, long ms) - { - return new TestPrefetchableTextFilesFirehoseFactory( - baseDir, - 1024, - 2048, - 2048, - 100, - 3, - 0, - 0, - ms - ); - } - - private static long computeTimeout(int maxRetry) - { - // See RetryUtils.nextRetrySleepMillis() - final double maxFuzzyMultiplier = 2.; - return (long) Math.min( - RetryUtils.MAX_SLEEP_MILLIS, - RetryUtils.BASE_SLEEP_MILLIS * Math.pow(2, maxRetry - 1) * maxFuzzyMultiplier - ); - } - - TestPrefetchableTextFilesFirehoseFactory( - File baseDir, - long prefetchTriggerThreshold, - long maxCacheCapacityBytes, - long maxFetchCapacityBytes, - int maxRetry, - int numOpenExceptions, - int numConnectionResets, - long sleepMillis - ) - { - this( - baseDir, - prefetchTriggerThreshold, - maxCacheCapacityBytes, - maxFetchCapacityBytes, - computeTimeout(maxRetry), - maxRetry, - numOpenExceptions, - numConnectionResets, - sleepMillis - ); - } - - TestPrefetchableTextFilesFirehoseFactory( - File baseDir, - long prefetchTriggerThreshold, - long maxCacheCapacityBytes, - long maxFetchCapacityBytes, - long fetchTimeout, - int maxRetry, - int numOpenExceptions, - int maxConnectionResets, - long sleepMillis - ) - { - super( - maxCacheCapacityBytes, - maxFetchCapacityBytes, - prefetchTriggerThreshold, - fetchTimeout, - maxRetry - ); - this.numOpenExceptions = numOpenExceptions; - this.maxConnectionResets = maxConnectionResets; - this.sleepMillis = sleepMillis; - this.baseDir = baseDir; - } - - @Override - protected Collection initObjects() - { - return FileUtils.listFiles( - Preconditions.checkNotNull(baseDir).getAbsoluteFile(), - TrueFileFilter.INSTANCE, - TrueFileFilter.INSTANCE - ); - } - - @Override - protected InputStream openObjectStream(File object) throws IOException - { - if (numOpenExceptions > 0) { - numOpenExceptions--; - throw new IOException("Exception for retry test"); - } - if (sleepMillis > 0) { - try { - Thread.sleep(sleepMillis); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - return maxConnectionResets > 0 ? - new TestInputStream(FileUtils.openInputStream(object), maxConnectionResets) : - FileUtils.openInputStream(object); - } - - @Override - protected InputStream wrapObjectStream(File object, InputStream stream) - { - return stream; - } - - @Override - protected Predicate getRetryCondition() - { - return e -> e instanceof IOException; - } - - @Override - protected InputStream openObjectStream(File object, long start) throws IOException - { - if (numOpenExceptions > 0) { - numOpenExceptions--; - throw new IOException("Exception for retry test"); - } - if (sleepMillis > 0) { - try { - Thread.sleep(sleepMillis); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - - final InputStream in = FileUtils.openInputStream(object); - in.skip(start); - - return maxConnectionResets > 0 ? new TestInputStream(in, maxConnectionResets) : in; - } - - private int readCount; - private int numConnectionResets; - - @Override - public FiniteFirehoseFactory withSplit(InputSplit split) - { - throw new UnsupportedOperationException(); - } - - private class TestInputStream extends InputStream - { - private static final int NUM_READ_COUNTS_BEFORE_ERROR = 10; - private final InputStream delegate; - private final int maxConnectionResets; - - TestInputStream( - InputStream delegate, - int maxConnectionResets - ) - { - this.delegate = delegate; - this.maxConnectionResets = maxConnectionResets; - } - - @Override - public int read() throws IOException - { - if (readCount++ % NUM_READ_COUNTS_BEFORE_ERROR == 0) { - if (numConnectionResets++ < maxConnectionResets) { - // Simulate connection reset - throw new SocketException("Test Connection reset"); - } - } - return delegate.read(); - } - - @Override - public int read(byte[] b, int off, int len) throws IOException - { - if (readCount++ % NUM_READ_COUNTS_BEFORE_ERROR == 0) { - if (numConnectionResets++ < maxConnectionResets) { - // Simulate connection reset - throw new SocketException("Test Connection reset"); - } - } - return delegate.read(b, off, len); - } - } - } -} diff --git a/server/src/main/java/org/apache/druid/guice/FirehoseModule.java b/server/src/main/java/org/apache/druid/guice/FirehoseModule.java index c95b0cd42ee..fe6461bf215 100644 --- a/server/src/main/java/org/apache/druid/guice/FirehoseModule.java +++ b/server/src/main/java/org/apache/druid/guice/FirehoseModule.java @@ -25,13 +25,8 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; import org.apache.druid.initialization.DruidModule; import org.apache.druid.segment.realtime.firehose.ClippedFirehoseFactory; -import org.apache.druid.segment.realtime.firehose.CombiningFirehoseFactory; import org.apache.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; import org.apache.druid.segment.realtime.firehose.FixedCountFirehoseFactory; -import org.apache.druid.segment.realtime.firehose.HttpFirehoseFactory; -import org.apache.druid.segment.realtime.firehose.InlineFirehoseFactory; -import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; -import org.apache.druid.segment.realtime.firehose.SqlFirehoseFactory; import org.apache.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory; import java.util.Collections; @@ -52,13 +47,8 @@ public class FirehoseModule implements DruidModule .registerSubtypes( new NamedType(ClippedFirehoseFactory.class, "clipped"), new NamedType(TimedShutoffFirehoseFactory.class, "timed"), - new NamedType(LocalFirehoseFactory.class, "local"), - new NamedType(HttpFirehoseFactory.class, "http"), new NamedType(EventReceiverFirehoseFactory.class, "receiver"), - new NamedType(CombiningFirehoseFactory.class, "combining"), - new NamedType(FixedCountFirehoseFactory.class, "fixedCount"), - new NamedType(SqlFirehoseFactory.class, "sql"), - new NamedType(InlineFirehoseFactory.class, "inline") + new NamedType(FixedCountFirehoseFactory.class, "fixedCount") ) ); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ClippedFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/ClippedFirehoseFactory.java index 9654d1f11b5..9ca595ce5c2 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ClippedFirehoseFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/ClippedFirehoseFactory.java @@ -34,6 +34,7 @@ import java.io.IOException; /** * Creates firehoses clipped to a particular time interval. Useful for enforcing min time, max time, and time windows. */ +@Deprecated public class ClippedFirehoseFactory implements FirehoseFactory { private final FirehoseFactory delegate; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/CombiningFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/CombiningFirehoseFactory.java deleted file mode 100644 index 6e61e19a1ea..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/CombiningFirehoseFactory.java +++ /dev/null @@ -1,160 +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.druid.segment.realtime.firehose; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.SplitHintSpec; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.java.util.emitter.EmittingLogger; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.util.Iterator; -import java.util.List; -import java.util.stream.Stream; - -/** - * Creates firehose that combines data from different Firehoses. Useful for ingesting data from multiple sources. - */ -public class CombiningFirehoseFactory implements FiniteFirehoseFactory> -{ - private static final EmittingLogger log = new EmittingLogger(CombiningFirehoseFactory.class); - - private final List delegateFactoryList; - - @JsonCreator - public CombiningFirehoseFactory( - @JsonProperty("delegates") List delegateFactoryList - ) - { - Preconditions.checkArgument(!delegateFactoryList.isEmpty()); - this.delegateFactoryList = delegateFactoryList; - } - - @Override - public Firehose connect(InputRowParser parser, File temporaryDirectory) - { - return new CombiningFirehose(parser, temporaryDirectory); - } - - @JsonProperty("delegates") - public List getDelegateFactoryList() - { - return delegateFactoryList; - } - - @Override - public boolean isSplittable() - { - return false; - } - - @Override - public Stream>> getSplits( - @Nullable SplitHintSpec splitHintSpec - ) - { - return Stream.of(new InputSplit<>(delegateFactoryList)); - } - - @Override - public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) - { - return 1; - } - - @Override - public FiniteFirehoseFactory> withSplit(InputSplit> split) - { - return new CombiningFirehoseFactory(split.get()); - } - - class CombiningFirehose implements Firehose - { - private final InputRowParser parser; - private final File temporaryDirectory; - private final Iterator firehoseFactoryIterator; - private volatile Firehose currentFirehose; - - CombiningFirehose(InputRowParser parser, File temporaryDirectory) - { - this.firehoseFactoryIterator = delegateFactoryList.iterator(); - this.parser = parser; - this.temporaryDirectory = temporaryDirectory; - nextFirehose(); - } - - private void nextFirehose() - { - if (firehoseFactoryIterator.hasNext()) { - try { - if (currentFirehose != null) { - currentFirehose.close(); - } - - currentFirehose = firehoseFactoryIterator.next().connect(parser, temporaryDirectory); - } - catch (IOException e) { - if (currentFirehose != null) { - try { - currentFirehose.close(); - } - catch (IOException e2) { - log.error(e, "Unable to close currentFirehose!"); - throw new RuntimeException(e2); - } - } - throw new RuntimeException(e); - } - } - } - - @Override - public boolean hasMore() throws IOException - { - return currentFirehose.hasMore(); - } - - @Nullable - @Override - public InputRow nextRow() throws IOException - { - InputRow rv = currentFirehose.nextRow(); - if (!currentFirehose.hasMore()) { - nextFirehose(); - } - return rv; - } - - @Override - public void close() throws IOException - { - currentFirehose.close(); - } - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java index 23c0a58e51d..4fad62a6d46 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java @@ -79,6 +79,7 @@ import java.util.concurrent.atomic.AtomicLong; * Builds firehoses that accept events through the {@link EventReceiver} interface. Can also register these * firehoses with an {@link ServiceAnnouncingChatHandlerProvider}. */ +@Deprecated public class EventReceiverFirehoseFactory implements FirehoseFactory>> { private static final EmittingLogger log = new EmittingLogger(EventReceiverFirehoseFactory.class); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactory.java deleted file mode 100644 index bbd797f2d3d..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactory.java +++ /dev/null @@ -1,165 +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.druid.segment.realtime.firehose; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.impl.HttpEntity; -import org.apache.druid.data.input.impl.HttpInputSource; -import org.apache.druid.data.input.impl.HttpInputSourceConfig; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; -import org.apache.druid.metadata.PasswordProvider; -import org.apache.druid.utils.CompressionUtils; - -import javax.annotation.Nullable; -import java.io.IOException; -import java.io.InputStream; -import java.net.URI; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Objects; - -public class HttpFirehoseFactory extends PrefetchableTextFilesFirehoseFactory -{ - private final List uris; - @Nullable - private final String httpAuthenticationUsername; - @Nullable - private final PasswordProvider httpAuthenticationPasswordProvider; - private final HttpInputSourceConfig inputSourceConfig; - - @JsonCreator - public HttpFirehoseFactory( - @JsonProperty("uris") List uris, - @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, - @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, - @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, - @JsonProperty("fetchTimeout") Long fetchTimeout, - @JsonProperty("maxFetchRetry") Integer maxFetchRetry, - @JsonProperty("httpAuthenticationUsername") @Nullable String httpAuthenticationUsername, - @JsonProperty("httpAuthenticationPassword") @Nullable PasswordProvider httpAuthenticationPasswordProvider, - @JacksonInject HttpInputSourceConfig inputSourceConfig - ) - { - super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry); - Preconditions.checkArgument(uris.size() > 0, "Empty URIs"); - HttpInputSource.throwIfInvalidProtocols(inputSourceConfig, uris); - this.uris = uris; - this.httpAuthenticationUsername = httpAuthenticationUsername; - this.httpAuthenticationPasswordProvider = httpAuthenticationPasswordProvider; - this.inputSourceConfig = inputSourceConfig; - } - - @Nullable - @JsonProperty - public String getHttpAuthenticationUsername() - { - return httpAuthenticationUsername; - } - - @Nullable - @JsonProperty("httpAuthenticationPassword") - public PasswordProvider getHttpAuthenticationPasswordProvider() - { - return httpAuthenticationPasswordProvider; - } - - @JsonProperty - public List getUris() - { - return uris; - } - - @Override - protected Collection initObjects() - { - return uris; - } - - @Override - protected InputStream openObjectStream(URI object) throws IOException - { - // A negative start value will ensure no bytes of the InputStream are skipped - return openObjectStream(object, 0); - } - - @Override - protected InputStream openObjectStream(URI object, long start) throws IOException - { - return HttpEntity.openInputStream(object, httpAuthenticationUsername, httpAuthenticationPasswordProvider, start); - } - - @Override - protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException - { - return CompressionUtils.decompress(stream, object.getPath()); - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - HttpFirehoseFactory that = (HttpFirehoseFactory) o; - return uris.equals(that.uris) && - Objects.equals(httpAuthenticationUsername, that.httpAuthenticationUsername) && - Objects.equals(httpAuthenticationPasswordProvider, that.httpAuthenticationPasswordProvider) && - inputSourceConfig.equals(that.inputSourceConfig); - } - - @Override - public int hashCode() - { - return Objects.hash(uris, httpAuthenticationUsername, httpAuthenticationPasswordProvider, inputSourceConfig); - } - - @Override - protected Predicate getRetryCondition() - { - return e -> e instanceof IOException; - } - - @Override - public FiniteFirehoseFactory withSplit(InputSplit split) - { - return new HttpFirehoseFactory( - Collections.singletonList(split.get()), - getMaxCacheCapacityBytes(), - getMaxFetchCapacityBytes(), - getPrefetchTriggerBytes(), - getFetchTimeout(), - getMaxFetchRetry(), - getHttpAuthenticationUsername(), - httpAuthenticationPasswordProvider, - inputSourceConfig - ); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehose.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehose.java deleted file mode 100644 index 75897f4d8a5..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehose.java +++ /dev/null @@ -1,91 +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.druid.segment.realtime.firehose; - -import org.apache.commons.io.IOUtils; -import org.apache.commons.io.LineIterator; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusRawValues; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.java.util.common.parsers.ParseException; - -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.nio.charset.Charset; -import java.util.NoSuchElementException; - -/** - * Firehose that produces data from its own spec - */ -public class InlineFirehose implements Firehose -{ - private final StringInputRowParser parser; - private final LineIterator lineIterator; - - InlineFirehose(String data, StringInputRowParser parser) - { - this.parser = parser; - - Charset charset = Charset.forName(parser.getEncoding()); - InputStream stream = new ByteArrayInputStream(data.getBytes(charset)); - lineIterator = IOUtils.lineIterator(stream, charset); - } - - @Override - public boolean hasMore() - { - return lineIterator.hasNext(); - } - - @Override - public InputRow nextRow() - { - return parser.parse(nextRaw()); - } - - private String nextRaw() - { - if (!hasMore()) { - throw new NoSuchElementException(); - } - - return lineIterator.next(); - } - - @Override - public InputRowListPlusRawValues nextRowWithRaw() - { - String raw = nextRaw(); - try { - return InputRowListPlusRawValues.of(parser.parse(raw), parser.parseString(raw)); - } - catch (ParseException e) { - return InputRowListPlusRawValues.of(parser.parseString(raw), e); - } - } - - @Override - public void close() throws IOException - { - lineIterator.close(); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseFactory.java deleted file mode 100644 index f609cbe87f3..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseFactory.java +++ /dev/null @@ -1,105 +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.druid.segment.realtime.firehose; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.SplitHintSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; - -import javax.annotation.Nullable; -import java.io.File; -import java.util.Objects; -import java.util.stream.Stream; - -/** - * Creates firehose that produces data inlined in its own spec - */ -public class InlineFirehoseFactory implements FiniteFirehoseFactory -{ - private final String data; - - @VisibleForTesting - @JsonCreator - public InlineFirehoseFactory(@JsonProperty("data") String data) - { - this.data = Preconditions.checkNotNull(data, "data"); - } - - @JsonProperty - public String getData() - { - return data; - } - - @Override - public Firehose connect(StringInputRowParser parser, @Nullable File temporaryDirectory) - { - return new InlineFirehose(data, parser); - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - InlineFirehoseFactory factory = (InlineFirehoseFactory) o; - return data.equals(factory.data); - } - - @Override - public int hashCode() - { - return Objects.hash(data); - } - - @Override - public boolean isSplittable() - { - return false; - } - - @Override - public Stream> getSplits(@Nullable SplitHintSpec splitHintSpec) - { - return Stream.of(new InputSplit<>(data)); - } - - @Override - public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) - { - return 1; - } - - @Override - public FiniteFirehoseFactory withSplit(InputSplit split) - { - return new InlineFirehoseFactory(split.get()); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactory.java deleted file mode 100644 index e11d1967b3b..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactory.java +++ /dev/null @@ -1,120 +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.druid.segment.realtime.firehose; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.filefilter.TrueFileFilter; -import org.apache.commons.io.filefilter.WildcardFileFilter; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.impl.AbstractTextFilesFirehoseFactory; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.utils.CompressionUtils; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Comparator; -import java.util.List; - -/** - * Firehose that reads data from files on local disk - */ -public class LocalFirehoseFactory extends AbstractTextFilesFirehoseFactory -{ - private final File baseDir; - private final String filter; - @Nullable - private final StringInputRowParser parser; - - @JsonCreator - public LocalFirehoseFactory( - @JsonProperty("baseDir") File baseDir, - @JsonProperty("filter") String filter, - // Backwards compatible - @Nullable @JsonProperty("parser") StringInputRowParser parser - ) - { - this.baseDir = baseDir; - this.filter = filter; - this.parser = parser; - } - - @JsonProperty - public File getBaseDir() - { - return baseDir; - } - - @JsonProperty - public String getFilter() - { - return filter; - } - - @JsonProperty - @Nullable - public StringInputRowParser getParser() - { - return parser; - } - - @Override - protected Collection initObjects() - { - final Collection files = FileUtils.listFiles( - Preconditions.checkNotNull(this.baseDir, "baseDir").getAbsoluteFile(), - new WildcardFileFilter(filter), - TrueFileFilter.INSTANCE - ); - - // Sort files for consistent ordering from run to run. - final List fileList = files instanceof List ? (List) files : new ArrayList<>(files); - fileList.sort(Comparator.naturalOrder()); - return fileList; - } - - @Override - protected InputStream openObjectStream(File object) throws IOException - { - return FileUtils.openInputStream(object); - } - - @Override - protected InputStream wrapObjectStream(File object, InputStream stream) throws IOException - { - return CompressionUtils.decompress(stream, object.getPath()); - } - - @Override - public FiniteFirehoseFactory withSplit(InputSplit split) - { - final File newFile = split.get(); - final File baseDir = newFile.getParentFile(); - final String filter = newFile.getName(); - return new LocalFirehoseFactory(baseDir, filter, parser); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/PrefetchSqlFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/PrefetchSqlFirehoseFactory.java deleted file mode 100644 index 683fef38a12..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/PrefetchSqlFirehoseFactory.java +++ /dev/null @@ -1,280 +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.druid.segment.realtime.firehose; - -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import org.apache.commons.io.LineIterator; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.SplitHintSpec; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.prefetch.CacheManager; -import org.apache.druid.data.input.impl.prefetch.FetchConfig; -import org.apache.druid.data.input.impl.prefetch.Fetcher; -import org.apache.druid.data.input.impl.prefetch.JsonIterator; -import org.apache.druid.data.input.impl.prefetch.ObjectOpenFunction; -import org.apache.druid.data.input.impl.prefetch.OpenObject; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.logger.Logger; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.stream.Stream; - -/** - * PrefetchSqlFirehoseFactory is an abstract firehose factory for reading prefetched sql resultset data. Regardless - * of whether prefetching is enabled or not, for each sql object the entire result set is fetched into a file in the local disk. - * This class defines prefetching as caching the resultsets into local disk in case multiple sql queries are present. - * When prefetching is enabled, the following functionalities are provided: - *

- *

- * - Caching: for the first call of {@link #connect(InputRowParser, File)}, it caches objects in a local disk - * up to maxCacheCapacityBytes. These caches are NOT deleted until the process terminates, and thus can be used for - * future reads. - *
- * - Fetching: when it reads all cached data, it fetches remaining objects into a local disk and reads data from - * them. For the performance reason, prefetch technique is used, that is, when the size of remaining fetched data is - * smaller than {@link FetchConfig#prefetchTriggerBytes}, a background prefetch thread automatically starts to fetch remaining - * objects. - *
- *

- *

- * This implementation aims to avoid maintaining a persistent connection to the database by prefetching the resultset into disk. - *
- * Prefetching can be turned on/off by setting maxFetchCapacityBytes. Depending on prefetching is enabled or - * disabled, the behavior of the firehose is different like below. - *

- *

- * 1. If prefetch is enabled this firehose can fetch input objects in background. - *
- * 2. When next() is called, it first checks that there are already fetched files in local storage. - *
- * 2.1 If exists, it simply chooses a fetched file and returns a {@link LineIterator} reading that file. - *
- * 2.2 If there is no fetched files in local storage but some objects are still remained to be read, the firehose - * fetches one of input objects in background immediately. Finally, the firehose returns an iterator of {@link JsonIterator} - * for deserializing the saved resultset. - *
- * 3. If prefetch is disabled, the firehose saves the resultset to file and returns an iterator of {@link JsonIterator} - * which directly reads the stream opened by {@link #openObjectStream}. If there is an IOException, it will throw it - * and the read will fail. - */ -public abstract class PrefetchSqlFirehoseFactory - implements FiniteFirehoseFactory>, T> -{ - private static final Logger LOG = new Logger(PrefetchSqlFirehoseFactory.class); - - private final FetchConfig fetchConfig; - private final CacheManager cacheManager; - private List objects; - private ObjectMapper objectMapper; - - - public PrefetchSqlFirehoseFactory( - Long maxCacheCapacityBytes, - Long maxFetchCapacityBytes, - Long prefetchTriggerBytes, - Long fetchTimeout, - ObjectMapper objectMapper - ) - { - this.fetchConfig = new FetchConfig( - maxCacheCapacityBytes, - maxFetchCapacityBytes, - prefetchTriggerBytes, - fetchTimeout, - 0 - ); - this.cacheManager = new CacheManager<>( - fetchConfig.getMaxCacheCapacityBytes() - ); - this.objectMapper = objectMapper; - } - - @JsonProperty - public long getMaxCacheCapacityBytes() - { - return cacheManager.getMaxCacheCapacityBytes(); - } - - @JsonProperty - public long getMaxFetchCapacityBytes() - { - return fetchConfig.getMaxFetchCapacityBytes(); - } - - @JsonProperty - public long getPrefetchTriggerBytes() - { - return fetchConfig.getPrefetchTriggerBytes(); - } - - @JsonProperty - public long getFetchTimeout() - { - return fetchConfig.getFetchTimeout(); - } - - @Override - public Firehose connect(InputRowParser> firehoseParser, @Nullable File temporaryDirectory) - { - if (objects == null) { - objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "objects")); - } - if (cacheManager.isEnabled() || fetchConfig.getMaxFetchCapacityBytes() > 0) { - Preconditions.checkNotNull(temporaryDirectory, "temporaryDirectory"); - Preconditions.checkArgument( - temporaryDirectory.exists(), - "temporaryDirectory[%s] does not exist", - temporaryDirectory - ); - Preconditions.checkArgument( - temporaryDirectory.isDirectory(), - "temporaryDirectory[%s] is not a directory", - temporaryDirectory - ); - } - - LOG.info("Create a new firehose for [%d] queries", objects.size()); - - // fetchExecutor is responsible for background data fetching - final ExecutorService fetchExecutor = Execs.singleThreaded("firehose_fetch_%d"); - final Fetcher fetcher = new SqlFetcher<>( - cacheManager, - objects, - fetchExecutor, - temporaryDirectory, - fetchConfig, - new ObjectOpenFunction() - { - @Override - public InputStream open(T object, File outFile) throws IOException - { - return openObjectStream(object, outFile); - } - - @Override - public InputStream open(T object) throws IOException - { - final File outFile = File.createTempFile("sqlresults_", null, temporaryDirectory); - return openObjectStream(object, outFile); - } - } - ); - - return new SqlFirehose( - new Iterator>>() - { - @Override - public boolean hasNext() - { - return fetcher.hasNext(); - } - - @Override - public JsonIterator> next() - { - if (!hasNext()) { - throw new NoSuchElementException(); - } - try { - TypeReference> type = new TypeReference>() - { - }; - final OpenObject openObject = fetcher.next(); - final InputStream stream = openObject.getObjectStream(); - return new JsonIterator<>(type, stream, openObject.getResourceCloser(), objectMapper); - } - catch (Exception ioe) { - throw new RuntimeException(ioe); - } - } - }, - firehoseParser, - () -> { - fetchExecutor.shutdownNow(); - try { - Preconditions.checkState(fetchExecutor.awaitTermination( - fetchConfig.getFetchTimeout(), - TimeUnit.MILLISECONDS - )); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new ISE("Failed to shutdown fetch executor during close"); - } - } - ); - } - - protected void initializeObjectsIfNeeded() - { - if (objects == null) { - objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "initObjects")); - } - } - - public List getObjects() - { - return objects; - } - - @Override - public Stream> getSplits(@Nullable SplitHintSpec splitHintSpec) - { - initializeObjectsIfNeeded(); - return getObjects().stream().map(InputSplit::new); - } - - @Override - public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) - { - initializeObjectsIfNeeded(); - return getObjects().size(); - } - - /** - * Open an input stream from the given object. The object is fetched into the file and an input - * stream to the file is provided. - * - * @param object an object to be read - * @param filename file to which the object is fetched into - * - * @return an input stream to the file - */ - protected abstract InputStream openObjectStream(T object, File filename) throws IOException; - - protected abstract Collection initObjects(); -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFetcher.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFetcher.java deleted file mode 100644 index 63385f7fdbb..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFetcher.java +++ /dev/null @@ -1,101 +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.druid.segment.realtime.firehose; - -import org.apache.druid.data.input.impl.prefetch.CacheManager; -import org.apache.druid.data.input.impl.prefetch.FetchConfig; -import org.apache.druid.data.input.impl.prefetch.Fetcher; -import org.apache.druid.data.input.impl.prefetch.ObjectOpenFunction; -import org.apache.druid.data.input.impl.prefetch.OpenObject; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.concurrent.ExecutorService; - - -/** - * A file fetcher used by {@link PrefetchSqlFirehoseFactory}. - * See the javadoc of {@link PrefetchSqlFirehoseFactory} for more details. - */ -public class SqlFetcher extends Fetcher -{ - private static final String FETCH_FILE_PREFIX = "sqlfetch-"; - - @Nullable - private final File temporaryDirectory; - - private final ObjectOpenFunction openObjectFunction; - - SqlFetcher( - CacheManager cacheManager, - List objects, - ExecutorService fetchExecutor, - @Nullable File temporaryDirectory, - FetchConfig fetchConfig, - ObjectOpenFunction openObjectFunction - ) - { - - super( - cacheManager, - objects, - fetchExecutor, - temporaryDirectory, - fetchConfig - ); - this.temporaryDirectory = temporaryDirectory; - this.openObjectFunction = openObjectFunction; - } - - /** - * Downloads the entire resultset object into a file. This avoids maintaining a - * persistent connection to the database. The retry is performed at the query execution layer. - * - * @param object sql query for which the resultset is to be downloaded - * @param outFile a file which the object data is stored - * - * @return size of downloaded resultset - */ - - @Override - protected long download(T object, File outFile) throws IOException - { - openObjectFunction.open(object, outFile); - return outFile.length(); - } - - /** - * Generates an instance of {@link OpenObject} for the given object. This is usually called - * when prefetching is disabled. The retry is performed at the query execution layer. - */ - - @Override - protected OpenObject generateOpenObject(T object) throws IOException - { - final File outFile = File.createTempFile(FETCH_FILE_PREFIX, null, temporaryDirectory); - return new OpenObject<>( - object, - openObjectFunction.open(object, outFile), - outFile::delete - ); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehose.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehose.java deleted file mode 100644 index 02c1dca387b..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehose.java +++ /dev/null @@ -1,95 +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.druid.segment.realtime.firehose; - -import com.google.common.collect.Iterators; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.prefetch.JsonIterator; -import org.apache.druid.segment.transform.TransformSpec; -import org.apache.druid.segment.transform.Transformer; - -import javax.annotation.Nullable; -import java.io.Closeable; -import java.io.IOException; -import java.util.Iterator; -import java.util.Map; - -public class SqlFirehose implements Firehose -{ - private final Iterator>> resultIterator; - private final MapInputRowParser parser; - private final Closeable closer; - @Nullable - private JsonIterator> lineIterator = null; - private final Transformer transformer; - - public SqlFirehose( - Iterator>> lineIterators, - InputRowParser parser, - Closeable closer - ) - { - this.resultIterator = lineIterators; - this.parser = new MapInputRowParser(parser.getParseSpec()); - // transformer is created from the original decorated parser (which should always be decorated) - this.transformer = TransformSpec.fromInputRowParser(parser).toTransformer(); - this.closer = closer; - } - - @Override - public boolean hasMore() - { - while ((lineIterator == null || !lineIterator.hasNext()) && resultIterator.hasNext()) { - lineIterator = getNextLineIterator(); - } - - return lineIterator != null && lineIterator.hasNext(); - } - - @Nullable - @Override - public InputRow nextRow() - { - assert lineIterator != null; - final Map mapToParse = lineIterator.next(); - return transformer.transform(Iterators.getOnlyElement(parser.parseBatch(mapToParse).iterator())); - } - - private JsonIterator> getNextLineIterator() - { - if (lineIterator != null) { - lineIterator = null; - } - - return resultIterator.next(); - } - - @Override - public void close() throws IOException - { - if (lineIterator != null) { - lineIterator.close(); - } - closer.close(); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactory.java deleted file mode 100644 index 0b5863d671d..00000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactory.java +++ /dev/null @@ -1,115 +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.druid.segment.realtime.firehose; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.guice.annotations.Smile; -import org.apache.druid.metadata.MetadataStorageConnectorConfig; -import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; -import org.apache.druid.metadata.input.SqlEntity; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -public class SqlFirehoseFactory extends PrefetchSqlFirehoseFactory -{ - @JsonProperty - private final List sqls; - @Nullable - @JsonProperty - private final MetadataStorageConnectorConfig connectorConfig; - private final ObjectMapper objectMapper; - @JsonProperty - private final SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector; - private final boolean foldCase; - - @JsonCreator - public SqlFirehoseFactory( - @JsonProperty("sqls") List sqls, - @JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes, - @JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes, - @JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes, - @JsonProperty("fetchTimeout") Long fetchTimeout, - @JsonProperty("foldCase") boolean foldCase, - @JsonProperty("database") SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector, - @JacksonInject @Smile ObjectMapper objectMapper - ) - { - super( - maxCacheCapacityBytes, - maxFetchCapacityBytes, - prefetchTriggerBytes, - fetchTimeout, - objectMapper - ); - Preconditions.checkArgument(sqls.size() > 0, "No SQL queries provided"); - - this.sqls = sqls; - this.objectMapper = objectMapper; - this.sqlFirehoseDatabaseConnector = Preconditions.checkNotNull( - sqlFirehoseDatabaseConnector, - "SQL Metadata Connector not configured!" - ); - this.foldCase = foldCase; - this.connectorConfig = null; - } - - @Override - protected InputStream openObjectStream(String sql, File fileName) throws IOException - { - SqlEntity.openCleanableFile(sql, sqlFirehoseDatabaseConnector, objectMapper, foldCase, fileName); - return new FileInputStream(fileName); - } - - @Override - protected Collection initObjects() - { - return sqls; - } - - @Override - public FiniteFirehoseFactory>, String> withSplit(InputSplit split) - { - return new SqlFirehoseFactory( - Collections.singletonList(split.get()), - getMaxCacheCapacityBytes(), - getMaxFetchCapacityBytes(), - getPrefetchTriggerBytes(), - getFetchTimeout(), - foldCase, - sqlFirehoseDatabaseConnector, - objectMapper - ); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java index 88d6d6ede22..9bfda42c8aa 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java @@ -43,6 +43,7 @@ import java.util.concurrent.TimeUnit; * Each {@link Firehose} created by this factory spins up and manages one thread for calling {@link Firehose#close()} * asynchronously at the specified {@link #shutoffTime}. */ +@Deprecated public class TimedShutoffFirehoseFactory implements FirehoseFactory { private static final EmittingLogger log = new EmittingLogger(FirehoseFactory.class); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/CombiningFirehoseFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/CombiningFirehoseFactoryTest.java deleted file mode 100644 index 14114701f2e..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/CombiningFirehoseFactoryTest.java +++ /dev/null @@ -1,174 +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.druid.segment.realtime.firehose; - -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.Row; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.parsers.ParseException; -import org.joda.time.DateTime; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Optional; - -public class CombiningFirehoseFactoryTest -{ - private CombiningFirehoseFactory combiningFirehoseFactory; - private List delegateFirehoses; - - @Before - public void setUp() - { - delegateFirehoses = Arrays.asList( - new ListFirehoseFactory(Arrays.asList(makeRow(1, 1), makeRow(2, 2))), - new ListFirehoseFactory(Arrays.asList(makeRow(3, 3), makeRow(4, 4), makeRow(5, 5))) - ); - combiningFirehoseFactory = new CombiningFirehoseFactory(delegateFirehoses); - } - - @Test - public void testCombiningfirehose() throws IOException - { - final Firehose firehose = combiningFirehoseFactory.connect(null, null); - for (int i = 1; i < 6; i++) { - Assert.assertTrue(firehose.hasMore()); - final InputRow inputRow = firehose.nextRow(); - Assert.assertEquals(i, inputRow.getTimestampFromEpoch()); - Assert.assertEquals(i, inputRow.getMetric("test").floatValue(), 0); - } - Assert.assertFalse(firehose.hasMore()); - } - - @Test - public void testFirehoseNotParallelizable() - { - Optional>> maybeFirehoseWithSplit = combiningFirehoseFactory.getSplits(null) - .findFirst(); - - Assert.assertTrue(maybeFirehoseWithSplit.isPresent()); - FiniteFirehoseFactory> firehoseWithSplit = combiningFirehoseFactory.withSplit( - maybeFirehoseWithSplit.get()); - Assert.assertTrue(firehoseWithSplit instanceof CombiningFirehoseFactory); - Assert.assertFalse(combiningFirehoseFactory.isSplittable()); - Assert.assertEquals(delegateFirehoses, ((CombiningFirehoseFactory) firehoseWithSplit).getDelegateFactoryList()); - } - - - private InputRow makeRow(final long timestamp, final float metricValue) - { - return new InputRow() - { - @Override - public List getDimensions() - { - return Collections.singletonList("testDim"); - } - - @Override - public long getTimestampFromEpoch() - { - return timestamp; - } - - @Override - public DateTime getTimestamp() - { - return DateTimes.utc(timestamp); - } - - @Override - public List getDimension(String dimension) - { - return new ArrayList<>(); - } - - @Override - public Number getMetric(String metric) - { - return metricValue; - } - - @Override - public Object getRaw(String dimension) - { - return null; - } - - @Override - public int compareTo(Row o) - { - return 0; - } - }; - } - - public static class ListFirehoseFactory implements FirehoseFactory - { - private final List rows; - - ListFirehoseFactory(List rows) - { - this.rows = rows; - } - - @Override - public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) throws ParseException - { - final Iterator iterator = rows.iterator(); - return new Firehose() - { - @Override - public boolean hasMore() - { - return iterator.hasNext(); - } - - @Nullable - @Override - public InputRow nextRow() - { - return iterator.next(); - } - - @Override - public void close() - { - // Do nothing - } - }; - } - - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactoryTest.java deleted file mode 100644 index 867c49778d5..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactoryTest.java +++ /dev/null @@ -1,145 +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.druid.segment.realtime.firehose; - -import com.fasterxml.jackson.databind.InjectableValues.Std; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import org.apache.druid.data.input.impl.HttpInputSourceConfig; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.metadata.DefaultPasswordProvider; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; - -import java.io.IOException; -import java.net.URI; - -public class HttpFirehoseFactoryTest -{ - @Rule - public ExpectedException expectedException = ExpectedException.none(); - - @Test - public void testSerde() throws IOException - { - final HttpInputSourceConfig inputSourceConfig = new HttpInputSourceConfig(null); - final ObjectMapper mapper = new DefaultObjectMapper(); - mapper.setInjectableValues(new Std().addValue( - HttpInputSourceConfig.class, - inputSourceConfig - )); - - final DefaultPasswordProvider pwProvider = new DefaultPasswordProvider("testPassword"); - final HttpFirehoseFactory factory = new HttpFirehoseFactory( - ImmutableList.of(URI.create("http://foo/bar"), URI.create("http://foo/bar2")), - 2048L, - 1024L, - 512L, - 100L, - 5, - "testUser", - pwProvider, - inputSourceConfig - ); - - final HttpFirehoseFactory outputFact = mapper.readValue( - mapper.writeValueAsString(factory), - HttpFirehoseFactory.class - ); - - Assert.assertEquals(factory, outputFact); - } - - @Test - public void testConstructorAllowsOnlyDefaultProtocols() - { - new HttpFirehoseFactory( - ImmutableList.of(URI.create("http:///")), - null, - null, - null, - null, - null, - null, - null, - new HttpInputSourceConfig(null) - ); - - new HttpFirehoseFactory( - ImmutableList.of(URI.create("https:///")), - null, - null, - null, - null, - null, - null, - null, - new HttpInputSourceConfig(null) - ); - - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("Only [http, https] protocols are allowed"); - new HttpFirehoseFactory( - ImmutableList.of(URI.create("my-protocol:///")), - null, - null, - null, - null, - null, - null, - null, - new HttpInputSourceConfig(null) - ); - } - - @Test - public void testConstructorAllowsOnlyCustomProtocols() - { - final HttpInputSourceConfig customConfig = new HttpInputSourceConfig(ImmutableSet.of("druid")); - new HttpFirehoseFactory( - ImmutableList.of(URI.create("druid:///")), - null, - null, - null, - null, - null, - null, - null, - customConfig - ); - - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("Only [druid] protocols are allowed"); - new HttpFirehoseFactory( - ImmutableList.of(URI.create("https:///")), - null, - null, - null, - null, - null, - null, - null, - customConfig - ); - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseFactoryTest.java deleted file mode 100644 index 053811e452b..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseFactoryTest.java +++ /dev/null @@ -1,129 +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.druid.segment.realtime.firehose; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.impl.CSVParseSpec; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.io.File; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.Arrays; -import java.util.List; -import java.util.Optional; - -@SuppressWarnings({"NullableProblems", "ConstantConditions"}) -public class InlineFirehoseFactoryTest -{ - private static final String DIMENSION_0 = "timestamp"; - private static final String DIMENSION_1 = "value"; - private static final List DIMENSIONS = Arrays.asList(DIMENSION_0, DIMENSION_1); - private static final String DELIMITER = ","; - private static final StringInputRowParser PARSER = new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec( - DIMENSION_0, - "auto", - null - ), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(DIMENSIONS)), - DELIMITER, - DIMENSIONS, - false, - 0 - ), - StandardCharsets.UTF_8.name() - ); - private static final File NO_TEMP_DIR = null; - private static final String TIMESTAMP = "0"; - private static final String VALUE = "a"; - private static final String DATA = TIMESTAMP + DELIMITER + VALUE; - - private InlineFirehoseFactory target; - - @Before - public void setUp() - { - target = new InlineFirehoseFactory(DATA); - } - - @Test - public void testInterfaceImplementation() - { - Assert.assertTrue(target instanceof FiniteFirehoseFactory); - Assert.assertFalse(target.isSplittable()); - Assert.assertEquals(1, target.getNumSplits(null)); - } - - @Test(expected = NullPointerException.class) - public void testContstructorDataRequired() - { - new InlineFirehoseFactory(null); - } - - @Test - public void testGetData() - { - Assert.assertEquals(DATA, target.getData()); - } - - @Test - public void testConnect() throws IOException - { - Firehose firehose = target.connect(PARSER, NO_TEMP_DIR); - InputRow row = firehose.nextRow(); - Assert.assertNotNull(row); - List values = row.getDimension(DIMENSION_1); - Assert.assertNotNull(values); - Assert.assertEquals(1, values.size()); - Assert.assertEquals(VALUE, values.get(0)); - } - - @Test - public void testForcedSplitAndClone() - { - Optional> inputSplitOptional = target.getSplits(null).findFirst(); - Assert.assertTrue(inputSplitOptional.isPresent()); - FiniteFirehoseFactory cloneWithSplit = target.withSplit(inputSplitOptional.get()); - Assert.assertTrue(cloneWithSplit instanceof InlineFirehoseFactory); - Assert.assertEquals(DATA, ((InlineFirehoseFactory) cloneWithSplit).getData()); - } - - @Test - public void testSerde() throws IOException - { - final ObjectMapper objectMapper = new DefaultObjectMapper(); - InlineFirehoseFactory factory = new InlineFirehoseFactory(DATA); - String serialized = objectMapper.writeValueAsString(factory); - InlineFirehoseFactory deserialized = objectMapper.readValue(serialized, InlineFirehoseFactory.class); - Assert.assertEquals(factory, deserialized); - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseTest.java deleted file mode 100644 index fc777b1021a..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseTest.java +++ /dev/null @@ -1,218 +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.druid.segment.realtime.firehose; - -import com.google.common.collect.Iterables; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusRawValues; -import org.apache.druid.data.input.impl.CSVParseSpec; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; -import java.nio.charset.Charset; -import java.nio.charset.StandardCharsets; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; - -@SuppressWarnings("ConstantConditions") -public class InlineFirehoseTest -{ - private static final String DIMENSION_0 = "timestamp"; - private static final String DIMENSION_1 = "value"; - private static final List DIMENSIONS = Arrays.asList(DIMENSION_0, DIMENSION_1); - private static final String DELIMITER = ","; - private static final Charset CHARSET = StandardCharsets.UTF_8; - private static final StringInputRowParser PARSER = new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec( - DIMENSION_0, - "auto", - null - ), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(DIMENSIONS)), - DELIMITER, - DIMENSIONS, - false, - 0 - ), - CHARSET.name() - ); - private static final String EMPTY = ""; - private static final String TIMESTAMP_0 = "0"; - private static final String VALUE_0 = "a"; - private static final String NOT_EMPTY = TIMESTAMP_0 + DELIMITER + VALUE_0; - private static final String PARSEABLE = NOT_EMPTY; - private static final String NOT_PARSEABLE = VALUE_0 + DELIMITER + TIMESTAMP_0; - private static final String TIMESTAMP_1 = "1"; - private static final String VALUE_1 = "b"; - private static final String LINE_0 = TIMESTAMP_0 + DELIMITER + VALUE_0; - private static final String LINE_1 = TIMESTAMP_1 + DELIMITER + VALUE_1; - private static final String MULTILINE = LINE_0 + "\n" + LINE_1; - - @Test - public void testHasMoreEmpty() - { - InlineFirehose target = create(EMPTY); - Assert.assertFalse(target.hasMore()); - } - - @Test - public void testHasMoreNotEmpty() - { - InlineFirehose target = create(NOT_EMPTY); - Assert.assertTrue(target.hasMore()); - } - - @Test(expected = NoSuchElementException.class) - public void testNextRowEmpty() - { - InlineFirehose target = create(EMPTY); - target.nextRow(); - } - - @Test - public void testNextRowNotEmpty() - { - InlineFirehose target = create(NOT_EMPTY); - InputRow row = target.nextRow(); - assertRowValue(VALUE_0, row); - } - - @Test(expected = NoSuchElementException.class) - public void testNextRowWithRawEmpty() - { - InlineFirehose target = create(EMPTY); - target.nextRowWithRaw(); - } - - @Test - public void testNextRowWithRawParseable() - { - final String data = PARSEABLE; - InlineFirehose target = create(data); - InputRowListPlusRawValues rowPlusRaw = target.nextRowWithRaw(); - - InputRow row = Iterables.getOnlyElement(rowPlusRaw.getInputRows()); - assertRowValue(VALUE_0, row); - - Map raw = rowPlusRaw.getRawValues(); - Map expected = new HashMap<>(); - expected.put("timestamp", TIMESTAMP_0); - expected.put("value", VALUE_0); - Assert.assertEquals(expected, raw); - - Assert.assertNull(rowPlusRaw.getParseException()); - } - - @Test - public void testNextRowWithRawNotParseable() - { - final String data = NOT_PARSEABLE; - InlineFirehose target = create(data); - InputRowListPlusRawValues rowPlusRaw = target.nextRowWithRaw(); - - Assert.assertNull(rowPlusRaw.getInputRows()); - - Map raw = rowPlusRaw.getRawValues(); - Map expected = new HashMap<>(); - expected.put("timestamp", VALUE_0); - expected.put("value", TIMESTAMP_0); - Assert.assertEquals(expected, raw); - - - Assert.assertNotNull(rowPlusRaw.getParseException()); - } - - @Test - public void testCloseOpen() throws IOException - { - InlineFirehose target = create(NOT_EMPTY); - target.close(); - try { - target.nextRow(); - Assert.fail("Should not be able to read from closed firehose"); - } - catch (NoSuchElementException ignored) { - } - } - - @Test - public void testCloseNotOpen() - { - InlineFirehose target = create(NOT_EMPTY); - try { - target.close(); - } - catch (IOException e) { - Assert.fail("Should be able to close an opened firehose"); - } - try { - target.close(); - } - catch (IOException e) { - Assert.fail("Should be able to close a closed firehose"); - } - } - - @Test - public void testMultiline() - { - InlineFirehose target = create(MULTILINE); - - // First line - Assert.assertTrue(target.hasMore()); - InputRow row0 = target.nextRow(); - assertRowValue(VALUE_0, row0); - - // Second line - InputRowListPlusRawValues rowPlusRaw = target.nextRowWithRaw(); - assertRowValue(VALUE_1, Iterables.getOnlyElement(rowPlusRaw.getInputRows())); - Map raw = rowPlusRaw.getRawValues(); - Map expected = new HashMap<>(); - expected.put("timestamp", TIMESTAMP_1); - expected.put("value", VALUE_1); - Assert.assertEquals(expected, raw); - Assert.assertNull(rowPlusRaw.getParseException()); - - Assert.assertFalse(target.hasMore()); - } - - private static InlineFirehose create(String data) - { - return new InlineFirehose(data, PARSER); - } - - private static void assertRowValue(String expected, InputRow row) - { - Assert.assertNotNull(row); - List values = row.getDimension(DIMENSION_1); - Assert.assertNotNull(values); - Assert.assertEquals(1, values.size()); - Assert.assertEquals(expected, values.get(0)); - } -} - diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java deleted file mode 100644 index 2e4a820c1b8..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java +++ /dev/null @@ -1,105 +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.druid.segment.realtime.firehose; - -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.Row; -import org.apache.druid.data.input.impl.CSVParseSpec; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.testing.InitializedNullHandlingTest; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.IOException; -import java.io.Writer; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; - -public class LocalFirehoseFactoryTest extends InitializedNullHandlingTest -{ - @Rule - public final TemporaryFolder temporaryFolder = new TemporaryFolder(); - - private LocalFirehoseFactory factory; - - @Before - public void setup() throws IOException - { - for (int i = 0; i < 5; i++) { - try (final Writer writer = - Files.newBufferedWriter(temporaryFolder.newFile("test_" + i).toPath(), StandardCharsets.UTF_8)) { - writer.write((20171225 + i) + "," + i + "th test file\n"); - } - } - - for (int i = 0; i < 5; i++) { - try (final Writer writer = - Files.newBufferedWriter(temporaryFolder.newFile("filtered_" + i).toPath(), StandardCharsets.UTF_8)) { - writer.write((20171225 + i) + "," + i + "th filtered file\n"); - } - } - - factory = new LocalFirehoseFactory(temporaryFolder.getRoot(), "test_*", null); - } - - @Test - public void testConnect() throws IOException - { - try (final Firehose firehose = factory.connect(new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec( - "timestamp", - "auto", - null - ), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a")) - ), - ",", - Arrays.asList("timestamp", "a"), - false, - 0 - ), - StandardCharsets.UTF_8.name() - ), null)) { - final List rows = new ArrayList<>(); - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - - Assert.assertEquals(5, rows.size()); - rows.sort(Comparator.comparing(Row::getTimestamp)); - for (int i = 0; i < 5; i++) { - final List dimVals = rows.get(i).getDimension("a"); - Assert.assertEquals(1, dimVals.size()); - Assert.assertEquals(i + "th test file", dimVals.get(0)); - } - } - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java deleted file mode 100644 index 38cc7582fab..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java +++ /dev/null @@ -1,182 +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.druid.segment.realtime.firehose; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.Row; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.metadata.TestDerbyConnector; -import org.apache.druid.metadata.input.SqlTestUtils; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.transform.TransformSpec; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -public class SqlFirehoseFactoryTest -{ - private static final String TABLE_NAME_1 = "FOOS_TABLE_1"; - private static final String TABLE_NAME_2 = "FOOS_TABLE_2"; - - @Rule - public TemporaryFolder test_dir = new TemporaryFolder(); - - @Rule - public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private final ObjectMapper mapper = TestHelper.makeSmileMapper(); - - private final InputRowParser parser = TransformSpec.NONE.decorate( - new MapInputRowParser( - new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "auto", null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a", "b")) - ) - ) - ) - ); - private TestDerbyConnector derbyConnector; - - private void assertNumRemainingCacheFiles(File firehoseTmpDir, int expectedNumFiles) - { - final String[] files = firehoseTmpDir.list(); - Assert.assertNotNull(files); - Assert.assertEquals(expectedNumFiles, files.length); - } - - private File createFirehoseTmpDir(String dirSuffix) throws IOException - { - return test_dir.newFolder(dirSuffix); - } - - @Test - public void testWithoutCacheAndFetch() throws Exception - { - derbyConnector = derbyConnectorRule.getConnector(); - SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); - final List expectedRows = testUtils.createTableWithRows(TABLE_NAME_1, 10); - final SqlFirehoseFactory factory = - new SqlFirehoseFactory( - SqlTestUtils.selectFrom(TABLE_NAME_1), - 0L, - 0L, - 0L, - 0L, - true, - testUtils.getDerbyFirehoseConnector(), - mapper - ); - - final List rows = new ArrayList<>(); - final File firehoseTmpDir = createFirehoseTmpDir("testWithoutCacheAndFetch"); - try (Firehose firehose = factory.connect(parser, firehoseTmpDir)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - - Assert.assertEquals(expectedRows, rows); - assertNumRemainingCacheFiles(firehoseTmpDir, 0); - testUtils.dropTable(TABLE_NAME_1); - } - - - @Test - public void testWithoutCache() throws IOException - { - derbyConnector = derbyConnectorRule.getConnector(); - SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); - final List expectedRows = testUtils.createTableWithRows(TABLE_NAME_1, 10); - final SqlFirehoseFactory factory = - new SqlFirehoseFactory( - SqlTestUtils.selectFrom(TABLE_NAME_1), - 0L, - null, - null, - null, - true, - testUtils.getDerbyFirehoseConnector(), - mapper - ); - - - final List rows = new ArrayList<>(); - final File firehoseTmpDir = createFirehoseTmpDir("testWithoutCache"); - try (Firehose firehose = factory.connect(parser, firehoseTmpDir)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - - Assert.assertEquals(expectedRows, rows); - assertNumRemainingCacheFiles(firehoseTmpDir, 0); - testUtils.dropTable(TABLE_NAME_1); - } - - - @Test - public void testWithCacheAndFetch() throws IOException - { - derbyConnector = derbyConnectorRule.getConnector(); - SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); - final List expectedRowsTable1 = testUtils.createTableWithRows(TABLE_NAME_1, 10); - final List expectedRowsTable2 = testUtils.createTableWithRows(TABLE_NAME_2, 10); - - final SqlFirehoseFactory factory = new - SqlFirehoseFactory( - SqlTestUtils.selectFrom(TABLE_NAME_1, TABLE_NAME_2), - null, - null, - 0L, - null, - true, - testUtils.getDerbyFirehoseConnector(), - mapper - ); - - final List rows = new ArrayList<>(); - final File firehoseTmpDir = createFirehoseTmpDir("testWithCacheAndFetch"); - try (Firehose firehose = factory.connect(parser, firehoseTmpDir)) { - while (firehose.hasMore()) { - rows.add(firehose.nextRow()); - } - } - - Assert.assertEquals(expectedRowsTable1, rows.subList(0, 10)); - Assert.assertEquals(expectedRowsTable2, rows.subList(10, 20)); - assertNumRemainingCacheFiles(firehoseTmpDir, 2); - testUtils.dropTable(TABLE_NAME_1); - testUtils.dropTable(TABLE_NAME_2); - - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseTest.java deleted file mode 100644 index 026b442881f..00000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseTest.java +++ /dev/null @@ -1,269 +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.druid.segment.realtime.firehose; - -import com.fasterxml.jackson.core.JsonGenerator; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.SerializerProvider; -import com.fasterxml.jackson.dataformat.smile.SmileFactory; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.data.input.impl.prefetch.JsonIterator; -import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.jackson.JacksonUtils; -import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.segment.transform.ExpressionTransform; -import org.apache.druid.segment.transform.TransformSpec; -import org.apache.druid.segment.transform.TransformingStringInputRowParser; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.io.Closeable; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; -import java.nio.charset.Charset; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -public class SqlFirehoseTest -{ - private static final TypeReference> TYPE_REF = new TypeReference>() - { - }; - private static File TEST_DIR; - private List> inputs; - private List fileList; - private InputRowParser parser; - private ObjectMapper objectMapper; - - - @Before - public void setup() throws IOException - { - TEST_DIR = File.createTempFile(SqlFirehose.class.getSimpleName(), "testDir"); - org.apache.commons.io.FileUtils.forceDelete(TEST_DIR); - FileUtils.mkdirp(TEST_DIR); - - final List> inputTexts = ImmutableList.of( - ImmutableMap.of("x", "foostring1", "timestamp", 2000), - ImmutableMap.of("x", "foostring2", "timestamp", 2000) - ); - List testFile = new ArrayList<>(); - this.objectMapper = new ObjectMapper(new SmileFactory()); - int i = 0; - for (Map m : inputTexts) { - File file = new File(TEST_DIR, "test_" + i++); - try (FileOutputStream fos = new FileOutputStream(file)) { - final JsonGenerator jg = objectMapper.getFactory().createGenerator(fos); - final SerializerProvider serializers = objectMapper.getSerializerProviderInstance(); - jg.writeStartArray(); - JacksonUtils.writeObjectUsingSerializerProvider(jg, serializers, m); - jg.writeEndArray(); - jg.close(); - testFile.add(new FileInputStream(file)); - } - } - - this.fileList = testFile; - parser = TransformSpec.NONE.decorate( - new MapInputRowParser( - new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "auto", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("x"))) - ) - ) - ); - - this.inputs = inputTexts; - } - - @Test - public void testFirehose() throws Exception - { - final TestCloseable closeable = new TestCloseable(); - List expectedResults = new ArrayList<>(); - for (Map map : inputs) { - expectedResults.add(map.get("x")); - } - final List>> lineIterators = - fileList.stream() - .map(s -> new JsonIterator>(TYPE_REF, s, closeable, objectMapper)) - .collect(Collectors.toList()); - - try (final SqlFirehose firehose = new SqlFirehose(lineIterators.iterator(), parser, closeable)) { - final List results = new ArrayList<>(); - - while (firehose.hasMore()) { - final InputRow inputRow = firehose.nextRow(); - if (inputRow == null) { - results.add(null); - } else { - results.add(inputRow.getDimension("x").get(0)); - } - } - - Assert.assertEquals(expectedResults, results); - } - } - - @Test - public void testFirehoseStringParser() throws Exception - { - final TestCloseable closeable = new TestCloseable(); - List expectedResults = new ArrayList<>(); - for (Map map : inputs) { - expectedResults.add(map.get("x")); - } - - final List>> lineIterators = - fileList.stream() - .map(s -> new JsonIterator>(TYPE_REF, s, closeable, objectMapper)) - .collect(Collectors.toList()); - - final InputRowParser stringParser = TransformSpec.NONE.decorate( - new StringInputRowParser( - new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "auto", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("x"))) - ), - Charset.defaultCharset().name() - ) - ); - try (final SqlFirehose firehose = new SqlFirehose(lineIterators.iterator(), stringParser, closeable)) { - final List results = new ArrayList<>(); - - while (firehose.hasMore()) { - final InputRow inputRow = firehose.nextRow(); - if (inputRow == null) { - results.add(null); - } else { - results.add(inputRow.getDimension("x").get(0)); - } - } - - Assert.assertEquals(expectedResults, results); - } - } - - @Test - public void testFirehoseTransformingParser() throws Exception - { - final TestCloseable closeable = new TestCloseable(); - List expectedResults = new ArrayList<>(); - for (Map map : inputs) { - expectedResults.add(map.get("x") + "foo"); - } - - final List>> lineIterators = - fileList.stream() - .map(s -> new JsonIterator>(TYPE_REF, s, closeable, objectMapper)) - .collect(Collectors.toList()); - - final InputRowParser stringParser = new TransformingStringInputRowParser( - new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "auto", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("x"))) - ), - Charset.defaultCharset().name(), - new TransformSpec( - null, - ImmutableList.of( - new ExpressionTransform("xfoo", "concat(x,'foo')", ExprMacroTable.nil()) - ) - ) - ); - - try (final SqlFirehose firehose = new SqlFirehose(lineIterators.iterator(), stringParser, closeable)) { - final List results = new ArrayList<>(); - - while (firehose.hasMore()) { - final InputRow inputRow = firehose.nextRow(); - if (inputRow == null) { - results.add(null); - } else { - results.add(inputRow.getDimension("xfoo").get(0)); - } - } - - Assert.assertEquals(expectedResults, results); - } - } - - @Test - public void testClose() throws IOException - { - File file = File.createTempFile("test", "", TEST_DIR); - final TestCloseable closeable = new TestCloseable(); - try (FileOutputStream fos = new FileOutputStream(file)) { - final JsonGenerator jg = objectMapper.getFactory().createGenerator(fos); - jg.writeStartArray(); - jg.writeEndArray(); - jg.close(); - } - - final JsonIterator> jsonIterator = new JsonIterator<>( - TYPE_REF, - new FileInputStream(file), - closeable, - objectMapper - ); - - final SqlFirehose firehose = new SqlFirehose( - ImmutableList.of(jsonIterator).iterator(), - parser, - closeable - ); - firehose.hasMore(); // initialize lineIterator - firehose.close(); - Assert.assertTrue(closeable.closed); - } - - @After - public void teardown() throws IOException - { - org.apache.commons.io.FileUtils.forceDelete(TEST_DIR); - } - - private static final class TestCloseable implements Closeable - { - private boolean closed; - - @Override - public void close() - { - closed = true; - } - } -} - diff --git a/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java b/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java index 1d309e0925c..3b920dfa656 100644 --- a/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java @@ -19,26 +19,21 @@ package org.apache.druid.cli.validate; -import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.github.rvesse.airline.Cli; import com.google.inject.Injector; -import org.apache.druid.guice.FirehoseModule; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.guice.GuiceInjectors; -import org.apache.druid.indexing.common.task.RealtimeIndexTask; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.realtime.FireDepartment; -import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; -import org.apache.druid.timeline.partition.NoneShardSpec; -import org.joda.time.Period; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -134,14 +129,10 @@ public class DruidJsonValidatorTest public void testTaskValidator() throws Exception { final ObjectMapper jsonMapper = new DefaultObjectMapper(); - for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) { - jsonMapper.registerModule(jacksonModule); - } - - final RealtimeIndexTask task = new RealtimeIndexTask( + final IndexTask task = new IndexTask( null, new TaskResource("rofl", 2), - new FireDepartment( + new IndexTask.IndexIngestionSpec( new DataSchema( "foo", null, @@ -150,28 +141,35 @@ public class DruidJsonValidatorTest null, jsonMapper ), - new RealtimeIOConfig( - new LocalFirehoseFactory(new File("lol"), "rofl", null), - (schema, config, metrics) -> null + new IndexTask.IndexIOConfig( + null, + new LocalInputSource(new File("lol"), "rofl"), + new JsonInputFormat(null, null, null, null, null), + false, + false ), - new RealtimeTuningConfig( - null, - 1, + new IndexTask.IndexTuningConfig( null, null, - new Period("PT10M"), + null, + 10, null, null, null, null, - 1, - NoneShardSpec.instance(), + null, + null, + new DynamicPartitionsSpec(10000, null), new IndexSpec(), - new IndexSpec(), - 0, - 0, - true, + null, + 3, + false, + null, + null, + null, + null, + null, null, null, null,