From 4b99c73ee5e01c57bc5fe686a68c80568e8f75a9 Mon Sep 17 00:00:00 2001
From: Clint Wylie
Date: Tue, 25 Jun 2024 03:23:02 -0700
Subject: [PATCH 1/4] Realtime cleanup and renaming changes: * `FireHydrant` is
now `PartialSegment`. This name much more clearly describes what this class
does, and with all the other fireman terminology removed it didn't even fit a
theme anymore. * `Sink` is now `AppendableSegment`. This name also much more
clearly describes what this class does, and is composed of `PartialSegments`
per the previous `FireHydrant` rename. * Additionally,
`SinkQuerySegmentWalker` -> `AppendableSegmentQuerySegmentWalker`, and
`SinkQueryRunner` -> `AppendableSegmentQueryRunner` * Remove `Firehose`,
`IngestSegmentFirehose` was only used by Hadoop indexing `DruidRecordReader`,
moved to internal class of `DruidRecordReader` as `SegmentReader` * Remove
`FirehoseFactory` and remaining implementations, after #16602 they were no
longer used * Moved things from `org.apache.druid.segment.realtime.sink` and
`org.apache.druid.segment.realtime.firehose` up to
`org.apache.druid.segment.realtime`.
---
docs/api-reference/tasks-api.md | 6 +-
docs/configuration/index.md | 1 -
docs/operations/metrics.md | 2 +-
.../k8s/overlord/common/K8sTestUtils.java | 1 -
.../DruidPeonClientIntegrationTest.java | 5 -
.../taskadapter/K8sTaskAdapterTest.java | 5 -
.../MultiContainerTaskAdapterTest.java | 5 -
.../SingleContainerTaskAdapterTest.java | 5 -
.../MaterializedViewSupervisorSpec.java | 2 +-
.../MaterializedViewSupervisorSpecTest.java | 4 +-
.../MaterializedViewSupervisorTest.java | 2 +-
.../kinesis/KinesisIndexTaskSerdeTest.java | 4 +-
.../indexing/IndexerControllerContext.java | 2 +-
.../client/ControllerChatHandler.java | 2 +-
.../indexing/client/WorkerChatHandler.java | 4 +-
.../druid/msq/test/MSQTestWorkerContext.java | 2 +-
.../hadoop/DatasourceRecordReader.java | 198 +++++-
.../indexer/BatchDeltaIngestionTest.java | 7 +-
.../indexer/hadoop/SegmentReaderTest.java | 34 +-
.../guice/IndexingServiceFirehoseModule.java | 49 --
.../druid/indexing/common/TaskToolbox.java | 2 +-
.../indexing/common/TaskToolboxFactory.java | 2 +-
.../stats/TaskRealtimeMetricsMonitor.java | 2 +-
.../indexing/common/task/CompactionTask.java | 1 -
.../indexing/common/task/HadoopIndexTask.java | 4 +-
.../druid/indexing/common/task/IndexTask.java | 34 +-
.../druid/indexing/common/task/Task.java | 9 -
...putSourceSplitParallelIndexTaskRunner.java | 4 -
.../parallel/LegacySinglePhaseSubTask.java | 3 -
.../batch/parallel/ParallelIndexIOConfig.java | 17 +-
.../parallel/ParallelIndexSupervisorTask.java | 7 +-
.../PartialDimensionCardinalityTask.java | 3 -
.../PartialDimensionDistributionTask.java | 3 -
.../PartialHashSegmentGenerateTask.java | 3 -
.../PartialRangeSegmentGenerateTask.java | 3 -
.../SinglePhaseParallelIndexTaskRunner.java | 4 -
.../batch/parallel/SinglePhaseSubTask.java | 5 +-
.../IndexTaskInputRowIteratorBuilder.java | 2 -
.../indexing/input/DruidSegmentReader.java | 2 +-
.../SeekableStreamIndexTask.java | 2 +-
.../SeekableStreamIndexTaskRunner.java | 2 +-
.../indexing/common/TaskToolboxTest.java | 2 +-
.../druid/indexing/common/TestFirehose.java | 118 ----
.../druid/indexing/common/TestIndexTask.java | 1 -
.../druid/indexing/common/TestUtils.java | 10 +-
.../common/task/BatchAppenderatorsTest.java | 2 +-
.../ClientCompactionTaskQuerySerdeTest.java | 4 +-
.../task/CompactionTaskParallelRunTest.java | 1 -
.../common/task/CompactionTaskRunTest.java | 4 +-
.../common/task/CompactionTaskTest.java | 4 +-
.../common/task/IndexIngestionSpecTest.java | 2 -
.../indexing/common/task/IndexTaskTest.java | 7 +-
.../common/task/IngestionTestBase.java | 7 +-
.../indexing/common/task/TaskSerdeTest.java | 28 +-
...bstractMultiPhaseParallelIndexingTest.java | 2 -
...stractParallelIndexSupervisorTaskTest.java | 4 +-
.../parallel/HashPartitionTaskKillTest.java | 2 -
...aseParallelIndexingWithNullColumnTest.java | 4 -
.../ParallelIndexSupervisorTaskKillTest.java | 3 -
...rallelIndexSupervisorTaskResourceTest.java | 2 -
.../ParallelIndexSupervisorTaskSerdeTest.java | 1 -
.../ParallelIndexSupervisorTaskTest.java | 5 +-
.../parallel/ParallelIndexTestingFactory.java | 2 +-
.../parallel/RangePartitionTaskKillTest.java | 1 -
.../SinglePhaseParallelIndexingTest.java | 6 -
.../parallel/SinglePhaseSubTaskSpecTest.java | 1 -
.../SingleTaskBackgroundRunnerTest.java | 2 +-
.../indexing/overlord/TaskLifecycleTest.java | 59 +-
.../indexing/overlord/TaskQueueTest.java | 1 -
.../overlord/TestTaskToolboxFactory.java | 2 +-
.../sampler/IndexTaskSamplerSpecTest.java | 2 -
.../SeekableStreamIndexTaskTestBase.java | 2 +-
.../indexing/worker/TaskAnnouncementTest.java | 2 +-
.../worker/WorkerTaskManagerTest.java | 2 +-
.../worker/WorkerTaskMonitorTest.java | 2 +-
integration-tests-ex/docs/conversion.md | 2 +-
integration-tests-ex/docs/dependencies.md | 2 +-
.../EventReceiverFirehoseTestClient.java | 181 -----
.../testing/guice/DruidTestModuleFactory.java | 2 -
.../org/apache/druid/data/input/Firehose.java | 97 ---
.../druid/data/input/FirehoseFactory.java | 82 ---
...ava => AppendableSegmentQueryRunners.java} | 4 +-
.../apache/druid/query/QueryResourceId.java | 4 +-
.../GroupByMergingQueryRunner.java | 9 +-
.../query/scan/ScanQueryRunnerFactory.java | 8 +-
.../BuildingHashBasedNumberedShardSpec.java | 4 +-
.../partition/BuildingNumberedShardSpec.java | 4 +-
publications/radstack/radstack.tex | 2 +-
.../druid/client/cache/CachePopulator.java | 3 +-
.../apache/druid/guice/FirehoseModule.java | 55 --
.../initialization/CoreInjectorBuilder.java | 2 -
.../rpc/indexing/SpecificTaskRetryPolicy.java | 2 +-
.../Sink.java => AppendableSegment.java} | 156 ++---
.../realtime/{firehose => }/ChatHandler.java | 2 +-
.../{firehose => }/ChatHandlerProvider.java | 2 +-
.../{firehose => }/ChatHandlerResource.java | 2 +-
.../realtime/{firehose => }/ChatHandlers.java | 2 +-
.../realtime/{sink => }/Committers.java | 2 +-
.../NoopChatHandlerProvider.java | 2 +-
.../{FireHydrant.java => PartialSegment.java} | 35 +-
...ence.java => PartialSegmentReference.java} | 26 +-
.../realtime/SegmentGenerationMetrics.java | 12 +-
.../ServiceAnnouncingChatHandlerProvider.java | 2 +-
.../WindowedStorageAdapter.java | 2 +-
... AppendableSegmentQuerySegmentWalker.java} | 92 +--
....java => AppendableSegmentSchemaUtil.java} | 65 +-
.../appenderator/AppenderatorImpl.java | 546 ++++++++-------
.../realtime/appenderator/Appenderators.java | 2 +-
.../appenderator/BatchAppenderator.java | 456 ++++++------
.../realtime/appenderator/Committed.java | 50 +-
.../realtime/appenderator/SegmentSchemas.java | 3 +-
.../appenderator/StreamAppenderator.java | 550 ++++++++-------
.../UnifiedIndexerAppenderatorsManager.java | 16 +-
.../firehose/ClippedFirehoseFactory.java | 81 ---
.../realtime/firehose/EventReceiver.java | 28 -
.../EventReceiverFirehoseFactory.java | 659 ------------------
.../firehose/FixedCountFirehoseFactory.java | 93 ---
.../firehose/IngestSegmentFirehose.java | 210 ------
.../realtime/firehose/PredicateFirehose.java | 89 ---
.../firehose/TimedShutoffFirehoseFactory.java | 139 ----
.../realtime/{firehose => }/package-info.java | 2 +-
.../BatchDataSegmentAnnouncer.java | 14 +-
.../coordination/DataSegmentAnnouncer.java | 4 +-
.../jetty/ChatHandlerServerModule.java | 2 +-
.../jetty/CliIndexerServerModule.java | 2 +-
.../TaskIdResponseHeaderFilterHolder.java | 2 +-
.../metrics/EventReceiverFirehoseMetric.java | 48 --
.../metrics/EventReceiverFirehoseMonitor.java | 90 ---
.../EventReceiverFirehoseRegister.java | 56 --
.../druid/server/metrics/MetricsModule.java | 1 -
.../druid/guice/FirehoseModuleTest.java | 93 ---
...nkTest.java => AppendableSegmentTest.java} | 85 ++-
.../ChatHandlerResourceTest.java | 2 +-
...drantTest.java => PartialSegmentTest.java} | 64 +-
...viceAnnouncingChatHandlerProviderTest.java | 2 +-
...a => AppendableSegmentSchemaUtilTest.java} | 42 +-
.../appenderator/AppenderatorsTest.java | 2 +-
...endableSegmentsBatchAppenderatorTest.java} | 160 ++---
...dableSegmentsBatchAppenderatorTester.java} | 14 +-
...mentsSinksBatchAppenderatorDriverTest.java | 4 +-
.../realtime/appenderator/CommittedTest.java | 22 +-
.../appenderator/StreamAppenderatorTest.java | 86 +--
.../EventReceiverFirehoseIdleTest.java | 136 ----
.../firehose/EventReceiverFirehoseTest.java | 442 ------------
.../java/org/apache/druid/cli/CliIndexer.java | 2 -
.../apache/druid/cli/CliMiddleManager.java | 6 +-
.../org/apache/druid/cli/CliOverlord.java | 6 +-
.../java/org/apache/druid/cli/CliPeon.java | 8 +-
.../cli/validate/DruidJsonValidator.java | 4 -
.../cli/validate/DruidJsonValidatorTest.java | 1 -
.../queries/multiple_partitions/q26.sql | 2 +-
.../queries/multiple_partitions/q27.sql | 2 +-
152 files changed, 1597 insertions(+), 4347 deletions(-)
rename server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java => indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/SegmentReaderTest.java (87%)
delete mode 100644 indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceFirehoseModule.java
delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/TestFirehose.java
delete mode 100644 integration-tests/src/main/java/org/apache/druid/testing/clients/EventReceiverFirehoseTestClient.java
delete mode 100644 processing/src/main/java/org/apache/druid/data/input/Firehose.java
delete mode 100644 processing/src/main/java/org/apache/druid/data/input/FirehoseFactory.java
rename processing/src/main/java/org/apache/druid/query/{SinkQueryRunners.java => AppendableSegmentQueryRunners.java} (90%)
delete mode 100644 server/src/main/java/org/apache/druid/guice/FirehoseModule.java
rename server/src/main/java/org/apache/druid/segment/realtime/{sink/Sink.java => AppendableSegment.java} (73%)
rename server/src/main/java/org/apache/druid/segment/realtime/{firehose => }/ChatHandler.java (95%)
rename server/src/main/java/org/apache/druid/segment/realtime/{firehose => }/ChatHandlerProvider.java (97%)
rename server/src/main/java/org/apache/druid/segment/realtime/{firehose => }/ChatHandlerResource.java (98%)
rename server/src/main/java/org/apache/druid/segment/realtime/{firehose => }/ChatHandlers.java (97%)
rename server/src/main/java/org/apache/druid/segment/realtime/{sink => }/Committers.java (96%)
rename server/src/main/java/org/apache/druid/segment/realtime/{firehose => }/NoopChatHandlerProvider.java (96%)
rename server/src/main/java/org/apache/druid/segment/realtime/{FireHydrant.java => PartialSegment.java} (86%)
rename server/src/main/java/org/apache/druid/segment/realtime/{sink/SinkSegmentReference.java => PartialSegmentReference.java} (68%)
rename server/src/main/java/org/apache/druid/segment/realtime/{firehose => }/ServiceAnnouncingChatHandlerProvider.java (98%)
rename server/src/main/java/org/apache/druid/segment/realtime/{firehose => }/WindowedStorageAdapter.java (96%)
rename server/src/main/java/org/apache/druid/segment/realtime/appenderator/{SinkQuerySegmentWalker.java => AppendableSegmentQuerySegmentWalker.java} (79%)
rename server/src/main/java/org/apache/druid/segment/realtime/appenderator/{SinkSchemaUtil.java => AppendableSegmentSchemaUtil.java} (58%)
delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/firehose/ClippedFirehoseFactory.java
delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiver.java
delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java
delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java
delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehose.java
delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/firehose/PredicateFirehose.java
delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java
rename server/src/main/java/org/apache/druid/segment/realtime/{firehose => }/package-info.java (94%)
delete mode 100644 server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseMetric.java
delete mode 100644 server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseMonitor.java
delete mode 100644 server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseRegister.java
delete mode 100644 server/src/test/java/org/apache/druid/guice/FirehoseModuleTest.java
rename server/src/test/java/org/apache/druid/segment/realtime/{sink/SinkTest.java => AppendableSegmentTest.java} (77%)
rename server/src/test/java/org/apache/druid/segment/realtime/{firehose => }/ChatHandlerResourceTest.java (97%)
rename server/src/test/java/org/apache/druid/segment/realtime/{FireHydrantTest.java => PartialSegmentTest.java} (76%)
rename server/src/test/java/org/apache/druid/segment/realtime/{firehose => }/ServiceAnnouncingChatHandlerProviderTest.java (99%)
rename server/src/test/java/org/apache/druid/segment/realtime/appenderator/{SinkSchemaUtilTest.java => AppendableSegmentSchemaUtilTest.java} (86%)
rename server/src/test/java/org/apache/druid/segment/realtime/appenderator/{ClosedSegmentsSinksBatchAppenderatorTest.java => ClosedSegmentsAppendableSegmentsBatchAppenderatorTest.java} (83%)
rename server/src/test/java/org/apache/druid/segment/realtime/appenderator/{ClosedSegmensSinksBatchAppenderatorTester.java => ClosedSegmentsAppendableSegmentsBatchAppenderatorTester.java} (94%)
delete mode 100644 server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseIdleTest.java
delete mode 100644 server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java
diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md
index dbd8fa7d5425..286fec939510 100644
--- a/docs/api-reference/tasks-api.md
+++ b/docs/api-reference/tasks-api.md
@@ -1042,12 +1042,12 @@ Host: http://ROUTER_IP:ROUTER_PORT
2023-07-03T22:11:17,902 INFO [task-runner-0-priority-0] org.apache.kafka.clients.consumer.KafkaConsumer - [Consumer clientId=consumer-kafka-supervisor-dcanhmig-1, groupId=kafka-supervisor-dcanhmig] Unsubscribed all topics or patterns and assigned partitions
2023-07-03T22:11:17,912 INFO [task-runner-0-priority-0] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Persisted rows[0] and (estimated) bytes[0]
2023-07-03T22:11:17,916 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-persist] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Flushed in-memory data with commit metadata [AppenderatorDriverMetadata{segments={}, lastSegmentIds={}, callerMetadata={nextPartitions=SeekableStreamEndSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}}}}] for segments:
- 2023-07-03T22:11:17,917 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-persist] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Persisted stats: processed rows: [0], persisted rows[0], sinks: [0], total fireHydrants (across sinks): [0], persisted fireHydrants (across sinks): [0]
+ 2023-07-03T22:11:17,917 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-persist] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Persisted stats: processed rows: [0], persisted rows[0], appendable segments: [0], total partial segments (across appendable segments): [0], persisted partial segments (across appendable segments): [0]
2023-07-03T22:11:17,919 INFO [task-runner-0-priority-0] org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver - Pushing [0] segments in background
2023-07-03T22:11:17,921 INFO [task-runner-0-priority-0] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Persisted rows[0] and (estimated) bytes[0]
2023-07-03T22:11:17,924 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-persist] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Flushed in-memory data with commit metadata [AppenderatorDriverMetadata{segments={}, lastSegmentIds={}, callerMetadata={nextPartitions=SeekableStreamStartSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}, exclusivePartitions=[]}, publishPartitions=SeekableStreamEndSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}}}}] for segments:
- 2023-07-03T22:11:17,924 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-persist] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Persisted stats: processed rows: [0], persisted rows[0], sinks: [0], total fireHydrants (across sinks): [0], persisted fireHydrants (across sinks): [0]
- 2023-07-03T22:11:17,925 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-merge] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Preparing to push (stats): processed rows: [0], sinks: [0], fireHydrants (across sinks): [0]
+ 2023-07-03T22:11:17,924 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-persist] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Persisted stats: processed rows: [0], persisted rows[0], appendable segments: [0], total partial segments (across appendable segments): [0], persisted partial segments (across appendable segments): [0]
+ 2023-07-03T22:11:17,925 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-merge] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Preparing to push (stats): processed rows: [0], appendable segments: [0], partial segments (across appendable segments): [0]
2023-07-03T22:11:17,925 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-merge] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Push complete...
2023-07-03T22:11:17,929 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-publish] org.apache.druid.indexing.seekablestream.SequenceMetadata - With empty segment set, start offsets [SeekableStreamStartSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}, exclusivePartitions=[]}] and end offsets [SeekableStreamEndSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}}] are the same, skipping metadata commit.
2023-07-03T22:11:17,930 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-publish] org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver - Published [0] segments with commit metadata [{nextPartitions=SeekableStreamStartSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}, exclusivePartitions=[]}, publishPartitions=SeekableStreamEndSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}}}]
diff --git a/docs/configuration/index.md b/docs/configuration/index.md
index b62ab9c0db81..6dc3522b5911 100644
--- a/docs/configuration/index.md
+++ b/docs/configuration/index.md
@@ -395,7 +395,6 @@ Metric monitoring is an essential part of Druid operations. The following monito
|`org.apache.druid.java.util.metrics.CgroupCpuSetMonitor`|Reports CPU core/HT and memory node allocations as per the `cpuset` cgroup.|
|`org.apache.druid.java.util.metrics.CgroupDiskMonitor`|Reports disk statistic as per the blkio cgroup.|
|`org.apache.druid.java.util.metrics.CgroupMemoryMonitor`|Reports memory statistic as per the memory cgroup.|
-|`org.apache.druid.server.metrics.EventReceiverFirehoseMonitor`|Reports how many events have been queued in the EventReceiverFirehose.|
|`org.apache.druid.server.metrics.HistoricalMetricsMonitor`|Reports statistics on Historical services. Available only on Historical services.|
|`org.apache.druid.server.metrics.SegmentStatsMonitor` | **EXPERIMENTAL** Reports statistics about segments on Historical services. Available only on Historical services. Not to be used when lazy loading is configured.|
|`org.apache.druid.server.metrics.QueryCountStatsMonitor`|Reports how many queries have been successful/failed/interrupted.|
diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md
index 1d37169684e1..3247e4e5cc7f 100644
--- a/docs/operations/metrics.md
+++ b/docs/operations/metrics.md
@@ -252,7 +252,7 @@ batch ingestion emit the following metrics. These metrics are deltas for each em
|`ingest/merge/time`|Milliseconds spent merging intermediate segments.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Depends on the configuration. Generally a few minutes at most.|
|`ingest/merge/cpu`|CPU time in Nanoseconds spent on merging intermediate segments.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Depends on the configuration. Generally a few minutes at most.|
|`ingest/handoff/count`|Number of handoffs that happened.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Varies. Generally greater than 0 once every segment granular period if cluster operating normally.|
-|`ingest/sink/count`|Number of sinks not handed off.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|1~3|
+|`ingest/sink/count`|Number of appendable segments not handed off.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|1~3|
|`ingest/events/messageGap`|Time gap in milliseconds between the latest ingested event timestamp and the current system timestamp of metrics emission. If the value is increasing but lag is low, Druid may not be receiving new data. This metric is reset as new tasks spawn up.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Greater than 0, depends on the time carried in event.|
|`ingest/notices/queueSize`|Number of pending notices to be processed by the coordinator.|`dataSource`, `tags`|Typically 0 and occasionally in lower single digits. Should not be a very high number. |
|`ingest/notices/time`|Milliseconds taken to process a notice by the supervisor.|`dataSource`, `tags`| < 1s |
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java
index 04b332aac850..b3fda99b222e 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java
@@ -79,7 +79,6 @@ public static Task getTask()
null
),
new IndexTask.IndexIOConfig(
- null,
new LocalInputSource(new File("lol"), "rofl"),
new NoopInputFormat(),
true,
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java
index 241b4d9fc68f..e2d97accc97f 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java
@@ -19,7 +19,6 @@
package org.apache.druid.k8s.overlord.taskadapter;
-import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import io.fabric8.kubernetes.api.model.Pod;
@@ -27,7 +26,6 @@
import io.fabric8.kubernetes.api.model.batch.v1.Job;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
-import org.apache.druid.guice.FirehoseModule;
import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.config.TaskConfigBuilder;
@@ -83,9 +81,6 @@ public void setup()
{
TestUtils utils = new TestUtils();
jsonMapper = utils.getTestObjectMapper();
- for (Module jacksonModule : new FirehoseModule().getJacksonModules()) {
- jsonMapper.registerModule(jacksonModule);
- }
jsonMapper.registerSubtypes(
new NamedType(ParallelIndexTuningConfig.class, "index_parallel"),
new NamedType(IndexTask.IndexTuningConfig.class, "index")
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java
index 102565efc35a..f8e7186a0263 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java
@@ -19,7 +19,6 @@
package org.apache.druid.k8s.overlord.taskadapter;
-import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.common.base.Joiner;
@@ -44,7 +43,6 @@
import org.apache.commons.lang.RandomStringUtils;
import org.apache.commons.lang.StringUtils;
import org.apache.druid.error.DruidException;
-import org.apache.druid.guice.FirehoseModule;
import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.config.TaskConfigBuilder;
@@ -101,9 +99,6 @@ public K8sTaskAdapterTest()
{
TestUtils utils = new TestUtils();
jsonMapper = utils.getTestObjectMapper();
- for (Module jacksonModule : new FirehoseModule().getJacksonModules()) {
- jsonMapper.registerModule(jacksonModule);
- }
jsonMapper.registerSubtypes(
new NamedType(ParallelIndexTuningConfig.class, "index_parallel"),
new NamedType(IndexTask.IndexTuningConfig.class, "index")
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java
index 58993b9a6a00..9308835d967f 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java
@@ -19,7 +19,6 @@
package org.apache.druid.k8s.overlord.taskadapter;
-import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.common.collect.ImmutableList;
@@ -28,7 +27,6 @@
import io.fabric8.kubernetes.api.model.batch.v1.Job;
import io.fabric8.kubernetes.client.KubernetesClient;
import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient;
-import org.apache.druid.guice.FirehoseModule;
import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.config.TaskConfigBuilder;
@@ -67,9 +65,6 @@ public void setup()
{
TestUtils utils = new TestUtils();
jsonMapper = utils.getTestObjectMapper();
- for (Module jacksonModule : new FirehoseModule().getJacksonModules()) {
- jsonMapper.registerModule(jacksonModule);
- }
jsonMapper.registerSubtypes(
new NamedType(ParallelIndexTuningConfig.class, "index_parallel"),
new NamedType(IndexTask.IndexTuningConfig.class, "index")
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java
index afc5299927fa..4cf9c3e4cbbc 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java
@@ -19,14 +19,12 @@
package org.apache.druid.k8s.overlord.taskadapter;
-import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import io.fabric8.kubernetes.api.model.Pod;
import io.fabric8.kubernetes.api.model.batch.v1.Job;
import io.fabric8.kubernetes.client.KubernetesClient;
import io.fabric8.kubernetes.client.server.mock.EnableKubernetesMockClient;
-import org.apache.druid.guice.FirehoseModule;
import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.config.TaskConfigBuilder;
@@ -66,9 +64,6 @@ public void setup()
{
TestUtils utils = new TestUtils();
jsonMapper = utils.getTestObjectMapper();
- for (Module jacksonModule : new FirehoseModule().getJacksonModules()) {
- jsonMapper.registerModule(jacksonModule);
- }
jsonMapper.registerSubtypes(
new NamedType(ParallelIndexTuningConfig.class, "index_parallel"),
new NamedType(IndexTask.IndexTuningConfig.class, "index")
diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java
index 7d59cb63ea5b..01039375259e 100644
--- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java
+++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java
@@ -47,7 +47,7 @@
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
-import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
+import org.apache.druid.segment.realtime.ChatHandlerProvider;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.timeline.DataSegment;
diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java
index 14bd59871253..65e71f626e8e 100644
--- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java
+++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java
@@ -40,8 +40,8 @@
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable;
import org.apache.druid.segment.TestHelper;
-import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
-import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider;
+import org.apache.druid.segment.realtime.ChatHandlerProvider;
+import org.apache.druid.segment.realtime.NoopChatHandlerProvider;
import org.apache.druid.server.security.AuthorizerMapper;
import org.easymock.EasyMock;
import org.hamcrest.CoreMatchers;
diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java
index 38f15a840dc2..ff1759195869 100644
--- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java
+++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java
@@ -52,7 +52,7 @@
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig;
import org.apache.druid.segment.metadata.SegmentSchemaManager;
-import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
+import org.apache.druid.segment.realtime.ChatHandlerProvider;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.timeline.DataSegment;
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java
index da04e3ab0a6d..70027a962b23 100644
--- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java
@@ -34,8 +34,8 @@
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
-import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
-import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider;
+import org.apache.druid.segment.realtime.ChatHandlerProvider;
+import org.apache.druid.segment.realtime.NoopChatHandlerProvider;
import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.Resource;
import org.apache.druid.server.security.ResourceAction;
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java
index 17ac82d736ba..993c2baacf80 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java
@@ -53,7 +53,7 @@
import org.apache.druid.query.QueryContext;
import org.apache.druid.rpc.ServiceClientFactory;
import org.apache.druid.rpc.indexing.OverlordClient;
-import org.apache.druid.segment.realtime.firehose.ChatHandler;
+import org.apache.druid.segment.realtime.ChatHandler;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/ControllerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/ControllerChatHandler.java
index bf3dd4a6bf14..d161b01bd0b6 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/ControllerChatHandler.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/ControllerChatHandler.java
@@ -22,7 +22,7 @@
import org.apache.druid.msq.exec.Controller;
import org.apache.druid.msq.indexing.IndexerResourcePermissionMapper;
import org.apache.druid.msq.rpc.ControllerResource;
-import org.apache.druid.segment.realtime.firehose.ChatHandler;
+import org.apache.druid.segment.realtime.ChatHandler;
import org.apache.druid.server.security.AuthorizerMapper;
public class ControllerChatHandler extends ControllerResource implements ChatHandler
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java
index 4ef6ab077cac..70d1ab11d380 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/WorkerChatHandler.java
@@ -33,8 +33,8 @@
import org.apache.druid.msq.kernel.WorkOrder;
import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
import org.apache.druid.msq.statistics.serde.ClusterByStatisticsSnapshotSerde;
-import org.apache.druid.segment.realtime.firehose.ChatHandler;
-import org.apache.druid.segment.realtime.firehose.ChatHandlers;
+import org.apache.druid.segment.realtime.ChatHandler;
+import org.apache.druid.segment.realtime.ChatHandlers;
import org.apache.druid.server.security.Action;
import org.apache.druid.utils.CloseableUtils;
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java
index ad05c20b5829..14f6f73b24ab 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java
@@ -44,7 +44,7 @@
import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.incremental.NoopRowIngestionMeters;
import org.apache.druid.segment.loading.DataSegmentPusher;
-import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider;
+import org.apache.druid.segment.realtime.NoopChatHandlerProvider;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.coordination.DataSegmentAnnouncer;
diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java
index 1e5da8af8ed6..d797f2954233 100644
--- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java
+++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java
@@ -20,36 +20,62 @@
package org.apache.druid.indexer.hadoop;
import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
import com.google.common.io.Closeables;
import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.MapBasedInputRow;
import org.apache.druid.data.input.Row;
+import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexer.HadoopDruidIndexerConfig;
import org.apache.druid.indexer.JobHelper;
+import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.FileUtils;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.guava.Yielder;
+import org.apache.druid.java.util.common.guava.Yielders;
import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
+import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.segment.BaseLongColumnValueSelector;
+import org.apache.druid.segment.BaseObjectColumnValueSelector;
+import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
-import org.apache.druid.segment.realtime.firehose.IngestSegmentFirehose;
-import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter;
+import org.apache.druid.segment.VirtualColumns;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.data.IndexedInts;
+import org.apache.druid.segment.filter.Filters;
+import org.apache.druid.segment.realtime.WindowedStorageAdapter;
+import org.apache.druid.segment.transform.TransformSpec;
+import org.apache.druid.segment.transform.Transformer;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import javax.annotation.Nullable;
+import java.io.Closeable;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
import java.util.List;
+import java.util.Map;
public class DatasourceRecordReader extends RecordReader
{
private static final Logger logger = new Logger(DatasourceRecordReader.class);
private DatasourceIngestionSpec spec;
- private IngestSegmentFirehose firehose;
+ private SegmentReader segmentReader;
private long rowNum;
private Row currRow;
@@ -108,7 +134,7 @@ public WindowedStorageAdapter apply(WindowedDataSegment segment)
}
);
- firehose = new IngestSegmentFirehose(
+ segmentReader = new SegmentReader(
adapters,
spec.getTransformSpec(),
spec.getDimensions(),
@@ -120,8 +146,8 @@ public WindowedStorageAdapter apply(WindowedDataSegment segment)
@Override
public boolean nextKeyValue()
{
- if (firehose.hasMore()) {
- currRow = firehose.nextRow();
+ if (segmentReader.hasMore()) {
+ currRow = segmentReader.nextRow();
rowNum++;
return true;
} else {
@@ -154,7 +180,7 @@ public float getProgress()
@Override
public void close() throws IOException
{
- Closeables.close(firehose, true);
+ Closeables.close(segmentReader, true);
for (QueryableIndex qi : indexes) {
Closeables.close(qi, true);
}
@@ -163,4 +189,162 @@ public void close() throws IOException
FileUtils.deleteDirectory(dir);
}
}
+
+ /**
+ * This can be replaced with DruidSegmentReader once indexing-service no longer depends on indexing-hadoop, but since
+ * the dependency is currently flipped, this is an inlined version of the old IngestSegmentFirehose
+ */
+ public static class SegmentReader implements Closeable
+ {
+ private final Transformer transformer;
+ private Yielder rowYielder;
+
+ public SegmentReader(
+ final List adapters,
+ final TransformSpec transformSpec,
+ final List dims,
+ final List metrics,
+ final DimFilter dimFilter
+ )
+ {
+ this.transformer = transformSpec.toTransformer();
+
+ Sequence rows = Sequences.concat(
+ Iterables.transform(
+ adapters,
+ new Function>()
+ {
+ @Nullable
+ @Override
+ public Sequence apply(WindowedStorageAdapter adapter)
+ {
+ return Sequences.concat(
+ Sequences.map(
+ adapter.getAdapter().makeCursors(
+ Filters.toFilter(dimFilter),
+ adapter.getInterval(),
+ VirtualColumns.EMPTY,
+ Granularities.ALL,
+ false,
+ null
+ ), new Function>()
+ {
+ @Nullable
+ @Override
+ public Sequence apply(final Cursor cursor)
+ {
+ final BaseLongColumnValueSelector timestampColumnSelector =
+ cursor.getColumnSelectorFactory().makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME);
+
+ final Map dimSelectors = new HashMap<>();
+ for (String dim : dims) {
+ final DimensionSelector dimSelector = cursor
+ .getColumnSelectorFactory()
+ .makeDimensionSelector(new DefaultDimensionSpec(dim, dim));
+ // dimSelector is null if the dimension is not present
+ if (dimSelector != null) {
+ dimSelectors.put(dim, dimSelector);
+ }
+ }
+
+ final Map metSelectors = new HashMap<>();
+ for (String metric : metrics) {
+ final BaseObjectColumnValueSelector metricSelector =
+ cursor.getColumnSelectorFactory().makeColumnValueSelector(metric);
+ metSelectors.put(metric, metricSelector);
+ }
+
+ return Sequences.simple(
+ new Iterable()
+ {
+ @Override
+ public Iterator iterator()
+ {
+ return new Iterator()
+ {
+ @Override
+ public boolean hasNext()
+ {
+ return !cursor.isDone();
+ }
+
+ @Override
+ public InputRow next()
+ {
+ final Map theEvent = Maps.newLinkedHashMap();
+ final long timestamp = timestampColumnSelector.getLong();
+ theEvent.put(TimestampSpec.DEFAULT_COLUMN, DateTimes.utc(timestamp));
+
+ for (Map.Entry dimSelector :
+ dimSelectors.entrySet()) {
+ final String dim = dimSelector.getKey();
+ final DimensionSelector selector = dimSelector.getValue();
+ final IndexedInts vals = selector.getRow();
+
+ int valsSize = vals.size();
+ if (valsSize == 1) {
+ final String dimVal = selector.lookupName(vals.get(0));
+ theEvent.put(dim, dimVal);
+ } else if (valsSize > 1) {
+ List dimVals = new ArrayList<>(valsSize);
+ for (int i = 0; i < valsSize; ++i) {
+ dimVals.add(selector.lookupName(vals.get(i)));
+ }
+ theEvent.put(dim, dimVals);
+ }
+ }
+
+ for (Map.Entry metSelector :
+ metSelectors.entrySet()) {
+ final String metric = metSelector.getKey();
+ final BaseObjectColumnValueSelector selector = metSelector.getValue();
+ Object value = selector.getObject();
+ if (value != null) {
+ theEvent.put(metric, value);
+ }
+ }
+ cursor.advance();
+ return new MapBasedInputRow(timestamp, dims, theEvent);
+ }
+
+ @Override
+ public void remove()
+ {
+ throw new UnsupportedOperationException("Remove Not Supported");
+ }
+ };
+ }
+ }
+ );
+ }
+ }
+ )
+ );
+ }
+ }
+ )
+ );
+ rowYielder = Yielders.each(rows);
+ }
+
+ public boolean hasMore()
+ {
+ return !rowYielder.isDone();
+ }
+
+ @Nullable
+ public InputRow nextRow()
+ {
+ final InputRow inputRow = rowYielder.get();
+ rowYielder = rowYielder.next(null);
+ return transformer.transform(inputRow);
+ }
+
+ @Override
+ public void close() throws IOException
+ {
+ rowYielder.close();
+ }
+
+ }
}
diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java
index ed8b8c0bb093..adab5c11d01b 100644
--- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java
+++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java
@@ -26,13 +26,13 @@
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import org.apache.commons.io.FileUtils;
-import org.apache.druid.data.input.Firehose;
import org.apache.druid.data.input.InputRow;
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.hll.HyperLogLogCollector;
+import org.apache.druid.indexer.hadoop.DatasourceRecordReader;
import org.apache.druid.indexer.hadoop.WindowedDataSegment;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.DateTimes;
@@ -49,8 +49,7 @@
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.loading.LocalDataSegmentPuller;
-import org.apache.druid.segment.realtime.firehose.IngestSegmentFirehose;
-import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter;
+import org.apache.druid.segment.realtime.WindowedStorageAdapter;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.DataSegment.PruneSpecsHolder;
@@ -404,7 +403,7 @@ private void testIngestion(
QueryableIndex index = INDEX_IO.loadIndex(tmpUnzippedSegmentDir);
StorageAdapter adapter = new QueryableIndexStorageAdapter(index);
- Firehose firehose = new IngestSegmentFirehose(
+ DatasourceRecordReader.SegmentReader firehose = new DatasourceRecordReader.SegmentReader(
ImmutableList.of(new WindowedStorageAdapter(adapter, windowedDataSegment.getInterval())),
TransformSpec.NONE,
expectedDimensions,
diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/SegmentReaderTest.java
similarity index 87%
rename from server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java
rename to indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/SegmentReaderTest.java
index 44f24f879a22..7f28ced09576 100644
--- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java
+++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/SegmentReaderTest.java
@@ -1,23 +1,4 @@
-/*
- * 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;
+package org.apache.druid.indexer.hadoop;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
@@ -47,6 +28,7 @@
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter;
import org.apache.druid.segment.incremental.OnheapIncrementalIndex;
+import org.apache.druid.segment.realtime.WindowedStorageAdapter;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
@@ -63,10 +45,8 @@
import java.util.Collection;
import java.util.List;
-/**
- */
@RunWith(Parameterized.class)
-public class IngestSegmentFirehoseTest
+public class SegmentReaderTest
{
private static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec(
ImmutableList.of(
@@ -107,7 +87,7 @@ public static Collection> constructorFeeder()
private final IndexIO indexIO;
private final IndexMerger indexMerger;
- public IngestSegmentFirehoseTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory)
+ public SegmentReaderTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory)
{
indexIO = TestHelper.getTestIndexIO();
indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory);
@@ -135,7 +115,7 @@ public void testReadFromIndexAndWriteAnotherIndex() throws Exception
) {
final StorageAdapter sa = new QueryableIndexStorageAdapter(qi);
final WindowedStorageAdapter wsa = new WindowedStorageAdapter(sa, sa.getInterval());
- final IngestSegmentFirehose firehose = new IngestSegmentFirehose(
+ final DatasourceRecordReader.SegmentReader firehose = new DatasourceRecordReader.SegmentReader(
ImmutableList.of(wsa, wsa),
TransformSpec.NONE,
ImmutableList.of("host", "spatial"),
@@ -168,7 +148,7 @@ public void testReadFromIndexAndWriteAnotherIndex() throws Exception
Assert.assertEquals(ImmutableList.of("visited_sum", "unique_hosts"), queryable.getAvailableMetrics());
// Do a spatial filter
- final IngestSegmentFirehose firehose2 = new IngestSegmentFirehose(
+ final DatasourceRecordReader.SegmentReader firehose2 = new DatasourceRecordReader.SegmentReader(
ImmutableList.of(new WindowedStorageAdapter(queryable, Intervals.of("2000/3000"))),
TransformSpec.NONE,
ImmutableList.of("host", "spatial"),
@@ -229,4 +209,4 @@ private void createTestIndex(File segmentDir) throws Exception
indexMerger.persist(index, segmentDir, IndexSpec.DEFAULT, null);
}
}
-}
+}
\ No newline at end of file
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
deleted file mode 100644
index 8e4952210997..000000000000
--- a/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceFirehoseModule.java
+++ /dev/null
@@ -1,49 +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.guice;
-
-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 org.apache.druid.segment.realtime.firehose.EventReceiverFirehoseFactory;
-
-import java.util.List;
-
-public class IndexingServiceFirehoseModule implements DruidModule
-{
- @Override
- public List extends Module> getJacksonModules()
- {
- return ImmutableList.of(
- new SimpleModule("IndexingServiceFirehoseModule")
- .registerSubtypes(
- new NamedType(EventReceiverFirehoseFactory.class, "receiver")
- )
- );
- }
-
- @Override
- public void configure(Binder binder)
- {
- }
-}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java
index 46de3064f033..0792bd8bb7b1 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java
@@ -60,7 +60,7 @@
import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager;
-import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
+import org.apache.druid.segment.realtime.ChatHandlerProvider;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.coordination.DataSegmentAnnouncer;
import org.apache.druid.server.coordination.DataSegmentServerAnnouncer;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java
index 669a30dc5de9..47a39e58873c 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java
@@ -60,7 +60,7 @@
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
-import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
+import org.apache.druid.segment.realtime.ChatHandlerProvider;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.coordination.DataSegmentAnnouncer;
import org.apache.druid.server.coordination.DataSegmentServerAnnouncer;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java
index 2fb11cd7a124..c2324a8e4810 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java
@@ -125,7 +125,7 @@ public boolean doMonitor(ServiceEmitter emitter)
emitter.emit(builder.setMetric("ingest/merge/time", metrics.mergeTimeMillis() - previousSegmentGenerationMetrics.mergeTimeMillis()));
emitter.emit(builder.setMetric("ingest/merge/cpu", metrics.mergeCpuTime() - previousSegmentGenerationMetrics.mergeCpuTime()));
emitter.emit(builder.setMetric("ingest/handoff/count", metrics.handOffCount() - previousSegmentGenerationMetrics.handOffCount()));
- emitter.emit(builder.setMetric("ingest/sink/count", metrics.sinkCount()));
+ emitter.emit(builder.setMetric("ingest/sink/count", metrics.appendableSegmentCount()));
long messageGap = metrics.messageGap();
if (messageGap >= 0) {
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java
index 81447f3fd5eb..3f33e8d7aa91 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java
@@ -765,7 +765,6 @@ private static ParallelIndexIOConfig createIoConfig(
}
return new ParallelIndexIOConfig(
- null,
new DruidInputSource(
dataSchema.getDataSource(),
interval,
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java
index bc9d5aca2b65..f1f96d8ca34d 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java
@@ -59,8 +59,8 @@
import org.apache.druid.segment.incremental.RowIngestionMeters;
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
-import org.apache.druid.segment.realtime.firehose.ChatHandler;
-import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
+import org.apache.druid.segment.realtime.ChatHandler;
+import org.apache.druid.segment.realtime.ChatHandlerProvider;
import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.server.security.Resource;
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 cc253f46a520..f55c92078666 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
@@ -21,8 +21,6 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonInclude;
-import com.fasterxml.jackson.annotation.JsonInclude.Include;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.fasterxml.jackson.databind.ObjectMapper;
@@ -34,7 +32,6 @@
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.FirehoseFactory;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.InputSource;
@@ -95,7 +92,7 @@
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
-import org.apache.druid.segment.realtime.firehose.ChatHandler;
+import org.apache.druid.segment.realtime.ChatHandler;
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.AuthorizerMapper;
@@ -312,9 +309,6 @@ public String getTaskAllocatorId()
@Override
public Set getInputSourceResources()
{
- if (ingestionSchema.getIOConfig().firehoseFactory != null) {
- throw getInputSecurityOnFirehoseUnsupportedError();
- }
return getIngestionSchema().getIOConfig().getInputSource() != null ?
getIngestionSchema().getIOConfig().getInputSource().getTypes()
.stream()
@@ -1124,7 +1118,6 @@ public IndexTuningConfig getTuningConfig()
public static class IndexIOConfig implements BatchIOConfig
{
- private final FirehoseFactory firehoseFactory;
private final InputSource inputSource;
private final AtomicReference inputSourceWithToolbox = new AtomicReference<>();
private final InputFormat inputFormat;
@@ -1133,42 +1126,19 @@ public static class IndexIOConfig implements BatchIOConfig
@JsonCreator
public IndexIOConfig(
- @Deprecated @JsonProperty("firehose") @Nullable FirehoseFactory firehoseFactory,
@JsonProperty("inputSource") @Nullable InputSource inputSource,
@JsonProperty("inputFormat") @Nullable InputFormat inputFormat,
@JsonProperty("appendToExisting") @Nullable Boolean appendToExisting,
@JsonProperty("dropExisting") @Nullable Boolean dropExisting
)
{
- Checks.checkOneNotNullOrEmpty(
- ImmutableList.of(new Property<>("firehose", firehoseFactory), new Property<>("inputSource", inputSource))
- );
- if (firehoseFactory != null && inputFormat != null) {
- throw new IAE("Cannot use firehose and inputFormat together. Try using inputSource instead of firehose.");
- }
- this.firehoseFactory = firehoseFactory;
+ Preconditions.checkNotNull(inputSource, "inputSource must not be null");
this.inputSource = inputSource;
this.inputFormat = inputFormat;
this.appendToExisting = appendToExisting == null ? BatchIOConfig.DEFAULT_APPEND_EXISTING : appendToExisting;
this.dropExisting = dropExisting == null ? BatchIOConfig.DEFAULT_DROP_EXISTING : dropExisting;
}
- // old constructor for backward compatibility
- @Deprecated
- public IndexIOConfig(FirehoseFactory firehoseFactory, @Nullable Boolean appendToExisting, @Nullable Boolean dropExisting)
- {
- this(firehoseFactory, null, null, appendToExisting, dropExisting);
- }
-
- @Nullable
- @JsonProperty("firehose")
- @JsonInclude(Include.NON_NULL)
- @Deprecated
- public FirehoseFactory getFirehoseFactory()
- {
- return firehoseFactory;
- }
-
@Nullable
@Override
@JsonProperty
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java
index 4fb4bb7852c0..96d4ceb69ddd 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java
@@ -165,15 +165,6 @@ default Set getInputSourceResources() throws UOE
));
}
- default UOE getInputSecurityOnFirehoseUnsupportedError()
- {
- throw new UOE(StringUtils.format(
- "Input source based security cannot be performed '%s' task because it uses firehose."
- + " Change the tasks configuration, or disable `isEnableInputSourceSecurity`",
- getType()
- ));
- }
-
/**
* Returns query runners for this task. If this task is not meant to answer queries over its datasource, this method
* should return null.
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 286325186b25..e6097678b5ae 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
@@ -19,7 +19,6 @@
package org.apache.druid.indexing.common.task.batch.parallel;
-import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.impl.SplittableInputSource;
@@ -81,14 +80,11 @@ final int estimateTotalNumSubTasks() throws IOException
final SubTaskSpec newTaskSpec(InputSplit split)
{
- final FirehoseFactory firehoseFactory;
final InputSource inputSource;
- firehoseFactory = null;
inputSource = baseInputSource.withSplit(split);
final ParallelIndexIngestionSpec subTaskIngestionSpec = new ParallelIndexIngestionSpec(
ingestionSchema.getDataSchema(),
new ParallelIndexIOConfig(
- firehoseFactory,
inputSource,
ingestionSchema.getIOConfig().getInputFormat(),
ingestionSchema.getIOConfig().isAppendToExisting(),
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/LegacySinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/LegacySinglePhaseSubTask.java
index 1daa99994274..27a242885c36 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/LegacySinglePhaseSubTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/LegacySinglePhaseSubTask.java
@@ -71,9 +71,6 @@ public String getType()
@Override
public Set getInputSourceResources()
{
- if (getIngestionSchema().getIOConfig().getFirehoseFactory() != null) {
- throw getInputSecurityOnFirehoseUnsupportedError();
- }
return getIngestionSchema().getIOConfig().getInputSource() != null ?
getIngestionSchema().getIOConfig().getInputSource().getTypes()
.stream()
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java
index c86446d91303..ff10e556de8f 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java
@@ -22,7 +22,6 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
-import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig;
@@ -37,26 +36,12 @@ public class ParallelIndexIOConfig extends IndexIOConfig
{
@JsonCreator
public ParallelIndexIOConfig(
- @JsonProperty("firehose") @Nullable FirehoseFactory firehoseFactory,
@JsonProperty("inputSource") @Nullable InputSource inputSource,
@JsonProperty("inputFormat") @Nullable InputFormat inputFormat,
@JsonProperty("appendToExisting") @Nullable Boolean appendToExisting,
@JsonProperty("dropExisting") @Nullable Boolean dropExisting
)
{
- super(firehoseFactory, inputSource, inputFormat, appendToExisting, dropExisting);
- }
-
- // old constructor for backward compatibility
- @Deprecated
- public ParallelIndexIOConfig(FirehoseFactory firehoseFactory, @Nullable Boolean appendToExisting)
- {
- this(firehoseFactory, null, null, appendToExisting, null);
- }
-
- @Deprecated
- public ParallelIndexIOConfig(FirehoseFactory firehoseFactory, @Nullable Boolean appendToExisting, boolean dropExisting)
- {
- this(firehoseFactory, null, null, appendToExisting, dropExisting);
+ super(inputSource, inputFormat, appendToExisting, dropExisting);
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
index 4ca0f1ff80d7..0b83a4ffa11b 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
@@ -77,8 +77,8 @@
import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
-import org.apache.druid.segment.realtime.firehose.ChatHandler;
-import org.apache.druid.segment.realtime.firehose.ChatHandlers;
+import org.apache.druid.segment.realtime.ChatHandler;
+import org.apache.druid.segment.realtime.ChatHandlers;
import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.server.security.Resource;
@@ -290,9 +290,6 @@ public String getType()
@Override
public Set getInputSourceResources()
{
- if (getIngestionSchema().getIOConfig().getFirehoseFactory() != null) {
- throw getInputSecurityOnFirehoseUnsupportedError();
- }
return getIngestionSchema().getIOConfig().getInputSource() != null ?
getIngestionSchema().getIOConfig().getInputSource().getTypes()
.stream()
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java
index 6724a2ebb90c..d75304f38c08 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTask.java
@@ -139,9 +139,6 @@ public String getType()
@Override
public Set getInputSourceResources()
{
- if (getIngestionSchema().getIOConfig().getFirehoseFactory() != null) {
- throw getInputSecurityOnFirehoseUnsupportedError();
- }
return getIngestionSchema().getIOConfig().getInputSource() != null ?
getIngestionSchema().getIOConfig().getInputSource().getTypes()
.stream()
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java
index cb361a204ea3..4bb395420b3f 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTask.java
@@ -180,9 +180,6 @@ public String getType()
@Override
public Set getInputSourceResources()
{
- if (getIngestionSchema().getIOConfig().getFirehoseFactory() != null) {
- throw getInputSecurityOnFirehoseUnsupportedError();
- }
return getIngestionSchema().getIOConfig().getInputSource() != null ?
getIngestionSchema().getIOConfig().getInputSource().getTypes()
.stream()
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java
index 383fc7afb2e0..4c224e396d18 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java
@@ -135,9 +135,6 @@ public String getType()
@Override
public Set getInputSourceResources()
{
- if (getIngestionSchema().getIOConfig().getFirehoseFactory() != null) {
- throw getInputSecurityOnFirehoseUnsupportedError();
- }
return getIngestionSchema().getIOConfig().getInputSource() != null ?
getIngestionSchema().getIOConfig().getInputSource().getTypes()
.stream()
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java
index cf7ae15a9a59..933df9ee7785 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java
@@ -153,9 +153,6 @@ public String getType()
@Override
public Set getInputSourceResources()
{
- if (getIngestionSchema().getIOConfig().getFirehoseFactory() != null) {
- throw getInputSecurityOnFirehoseUnsupportedError();
- }
return getIngestionSchema().getIOConfig().getInputSource() != null ?
getIngestionSchema().getIOConfig().getInputSource().getTypes()
.stream()
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 ce6aee98af35..a2a29b3cdd3e 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
@@ -21,7 +21,6 @@
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.InputSource;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.impl.SplittableInputSource;
@@ -174,9 +173,7 @@ int estimateTotalNumSubTasks() throws IOException
@VisibleForTesting
SubTaskSpec newTaskSpec(InputSplit split)
{
- final FirehoseFactory firehoseFactory;
final InputSource inputSource;
- firehoseFactory = null;
inputSource = baseInputSource.withSplit(split);
final Map subtaskContext = new HashMap<>(getContext());
@@ -187,7 +184,6 @@ SubTaskSpec newTaskSpec(InputSplit split)
new ParallelIndexIngestionSpec(
ingestionSchema.getDataSchema(),
new ParallelIndexIOConfig(
- firehoseFactory,
inputSource,
ingestionSchema.getIOConfig().getInputFormat(),
ingestionSchema.getIOConfig().isAppendToExisting(),
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 cd05e50ff1c8..9d6756936ed0 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
@@ -66,7 +66,7 @@
import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
-import org.apache.druid.segment.realtime.firehose.ChatHandler;
+import org.apache.druid.segment.realtime.ChatHandler;
import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.server.security.Resource;
@@ -198,9 +198,6 @@ public String getType()
@Override
public Set getInputSourceResources()
{
- if (getIngestionSchema().getIOConfig().getFirehoseFactory() != null) {
- throw getInputSecurityOnFirehoseUnsupportedError();
- }
return getIngestionSchema().getIOConfig().getInputSource() != null ?
getIngestionSchema().getIOConfig().getInputSource().getTypes()
.stream()
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java
index 89afdbb64d05..3e3d653e3c32 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java
@@ -19,7 +19,6 @@
package org.apache.druid.indexing.common.task.batch.parallel.iterator;
-import org.apache.druid.data.input.Firehose;
import org.apache.druid.data.input.HandlingInputRowIterator;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.java.util.common.parsers.CloseableIterator;
@@ -34,7 +33,6 @@ public interface IndexTaskInputRowIteratorBuilder
/**
* @param granularitySpec {@link GranularitySpec} for the {@link org.apache.druid.segment.indexing.DataSchema}
- * associated with the {@link Firehose}.
*/
IndexTaskInputRowIteratorBuilder granularitySpec(GranularitySpec granularitySpec);
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java
index d048cf4d8b91..42618556d003 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java
@@ -56,7 +56,7 @@
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.data.IndexedInts;
import org.apache.druid.segment.filter.Filters;
-import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter;
+import org.apache.druid.segment.realtime.WindowedStorageAdapter;
import org.apache.druid.utils.CloseableUtils;
import org.apache.druid.utils.CollectionUtils;
import org.joda.time.Interval;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java
index f8e78bb711d6..9d2d76cd20ed 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java
@@ -52,7 +52,7 @@
import org.apache.druid.segment.realtime.SegmentGenerationMetrics;
import org.apache.druid.segment.realtime.appenderator.Appenderator;
import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver;
-import org.apache.druid.segment.realtime.firehose.ChatHandler;
+import org.apache.druid.segment.realtime.ChatHandler;
import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.timeline.partition.NumberedPartialShardSpec;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java
index 2fbad2c22240..79683de72d7a 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java
@@ -87,7 +87,7 @@
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
import org.apache.druid.segment.realtime.appenderator.StreamAppenderator;
import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver;
-import org.apache.druid.segment.realtime.firehose.ChatHandler;
+import org.apache.druid.segment.realtime.ChatHandler;
import org.apache.druid.server.security.Access;
import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.AuthorizerMapper;
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java
index e1ac9482436b..39aa41cabe29 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java
@@ -53,7 +53,7 @@
import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager;
-import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider;
+import org.apache.druid.segment.realtime.NoopChatHandlerProvider;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.coordination.DataSegmentAnnouncer;
import org.apache.druid.server.coordination.DataSegmentServerAnnouncer;
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestFirehose.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestFirehose.java
deleted file mode 100644
index 1572373151bf..000000000000
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestFirehose.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.indexing.common;
-
-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.InputRowParser;
-import org.apache.druid.data.input.impl.StringInputRowParser;
-import org.apache.druid.java.util.common.parsers.ParseException;
-
-import java.io.File;
-import java.util.ArrayDeque;
-import java.util.Collections;
-import java.util.Deque;
-import java.util.List;
-import java.util.Optional;
-
-public class TestFirehose implements Firehose
-{
- public static class TestFirehoseFactory implements FirehoseFactory
- {
- private boolean waitForClose = true;
-
- @Override
- @SuppressWarnings("unchecked")
- public Firehose connect(InputRowParser parser, File temporaryDirectory) throws ParseException
- {
- return new TestFirehose(parser, waitForClose, Collections.emptyList());
- }
- }
-
- public static final String FAIL_DIM = "__fail__";
-
- private final Deque> queue = new ArrayDeque<>();
-
- private InputRowParser parser;
- private boolean closed;
-
- private TestFirehose(InputRowParser parser, boolean waitForClose, List
- */
-@Deprecated
-@ExtensionPoint
-public interface Firehose extends Closeable
-{
- /**
- * Returns whether there are more rows to process. This is used to indicate that another item is immediately
- * available via {@link #nextRow()}. Thus, if the stream is still available but there are no new messages on it, this
- * call should block until a new message is available.
- *
- * If something happens such that the stream is no longer available, this should return false.
- *
- * @return true if and when there is another row available, false if the stream has dried up
- */
- boolean hasMore() throws IOException;
-
- /**
- * The next row available. Should only be called if hasMore returns true.
- * The return value can be null which means the caller must skip this row.
- *
- * @return The next row
- */
- @Nullable
- InputRow nextRow() throws IOException;
-
- /**
- * Returns an {@link InputRowListPlusRawValues} object containing the InputRow plus the raw, unparsed data corresponding to
- * the next row available. Used in the sampler to provide the caller with information to assist in configuring a parse
- * spec. If a ParseException is thrown by the parser, it should be caught and returned in the InputRowListPlusRawValues so
- * we will be able to provide information on the raw row which failed to be parsed. Should only be called if hasMore
- * returns true.
- *
- * @return an InputRowListPlusRawValues which may contain any of: an InputRow, map of the raw data, or a ParseException
- */
- @Deprecated
- default InputRowListPlusRawValues nextRowWithRaw() throws IOException
- {
- try {
- return InputRowListPlusRawValues.of(nextRow(), null);
- }
- catch (ParseException e) {
- return InputRowListPlusRawValues.of(null, e);
- }
- }
-
- /**
- * Closes the "ingestion side" of the Firehose, potentially concurrently with calls to {@link #hasMore()} and {@link
- * #nextRow()} being made from a different thread. {@link #hasMore()} and {@link #nextRow()}
- * continue to work after close(), but since the ingestion side is closed rows will eventually run out.
- */
- @Override
- void close() throws IOException;
-}
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
deleted file mode 100644
index 6ec8657d435a..000000000000
--- a/processing/src/main/java/org/apache/druid/data/input/FirehoseFactory.java
+++ /dev/null
@@ -1,82 +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 com.fasterxml.jackson.annotation.JsonTypeInfo;
-import org.apache.druid.data.input.impl.InputRowParser;
-import org.apache.druid.guice.annotations.ExtensionPoint;
-import org.apache.druid.java.util.common.parsers.ParseException;
-
-import javax.annotation.Nullable;
-import java.io.File;
-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
-{
- /**
- * Initialization method that connects up the fire hose. If this method returns successfully it should be safe to
- * call hasMore() on the returned Firehose (which might subsequently block).
- *
- * 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.
- *
- * @param parser an input row parser
- */
- @Deprecated
- default Firehose connect(T parser) throws IOException, ParseException
- {
- return connect(parser, null);
- }
-
- /**
- * Initialization method that connects up the fire hose. If this method returns successfully it should be safe to
- * call hasMore() on the returned Firehose (which might subsequently block).
- *
- * 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.
- *
- * @param parser an input row parser
- * @param temporaryDirectory a directory where temporary files are stored
- */
- default Firehose connect(T parser, @Nullable File temporaryDirectory) throws IOException, ParseException
- {
- return connect(parser);
- }
-
- @SuppressWarnings("unused")
- @JsonIgnore
- default boolean isSplittable()
- {
- return false;
- }
-}
diff --git a/processing/src/main/java/org/apache/druid/query/SinkQueryRunners.java b/processing/src/main/java/org/apache/druid/query/AppendableSegmentQueryRunners.java
similarity index 90%
rename from processing/src/main/java/org/apache/druid/query/SinkQueryRunners.java
rename to processing/src/main/java/org/apache/druid/query/AppendableSegmentQueryRunners.java
index 9cec39a61b13..f7c58d6336f3 100644
--- a/processing/src/main/java/org/apache/druid/query/SinkQueryRunners.java
+++ b/processing/src/main/java/org/apache/druid/query/AppendableSegmentQueryRunners.java
@@ -24,11 +24,11 @@
import java.util.Iterator;
-public class SinkQueryRunners implements Iterable>
+public class AppendableSegmentQueryRunners implements Iterable>
{
Iterable>> runners;
- public SinkQueryRunners(Iterable>> runners)
+ public AppendableSegmentQueryRunners(Iterable>> runners)
{
this.runners = runners;
}
diff --git a/processing/src/main/java/org/apache/druid/query/QueryResourceId.java b/processing/src/main/java/org/apache/druid/query/QueryResourceId.java
index da988de4b793..ce6f682bda5f 100644
--- a/processing/src/main/java/org/apache/druid/query/QueryResourceId.java
+++ b/processing/src/main/java/org/apache/druid/query/QueryResourceId.java
@@ -48,7 +48,7 @@
* there are three queryable servers (classes are not linkable from this method):
* 1. {@link org.apache.druid.server.ClientQuerySegmentWalker} - For brokers
* 2. {@link org.apache.druid.server.coordination.ServerManager} - For historicals
- * 3. {@link org.apache.druid.segment.realtime.appenderator.SinkQuerySegmentWalker} - For peons & indexer's tasks
+ * 3. {@link org.apache.druid.segment.realtime.appenderator.AppendableSegmentQuerySegmentWalker} - For peons & indexer's tasks
*
* These three classes are one of the first places the query reaches when it begins processing, therefore it is
* guaranteed that if the resource id is allotted at only these places, no one will overwrite the resource id
@@ -68,7 +68,7 @@
*
* 3. {@link org.apache.druid.server.coordination.ServerManager} Allocates the query resource id on the historicals
*
- * 4. {@link org.apache.druid.segment.realtime.appenderator.SinkQuerySegmentWalker} Allocates the query resource id on the peons
+ * 4. {@link org.apache.druid.segment.realtime.appenderator.AppendableSegmentQuerySegmentWalker} Allocates the query resource id on the peons
* (MMs) and indexers
*
* 5. {@link org.apache.druid.server.ResourceIdPopulatingQueryRunner} Populates the query resource id. ({@link org.apache.druid.server.ClientQuerySegmentWalker}
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunner.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunner.java
index 6e2064d5f1f1..9fa4c5f99bbd 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunner.java
@@ -136,10 +136,11 @@ public Sequence run(final QueryPlus queryPlus, final Respo
final GroupByQueryConfig querySpecificConfig = config.withOverrides(query);
// CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION is here because realtime servers use nested mergeRunners calls
- // (one for the entire query and one for each sink). We only want the outer call to actually do merging with a
- // merge buffer, otherwise the query will allocate too many merge buffers. This is potentially sub-optimal as it
- // will involve materializing the results for each sink before starting to feed them into the outer merge buffer.
- // I'm not sure of a better way to do this without tweaking how realtime servers do queries.
+ // (one for the entire query and one for each appendable segment). We only want the outer call to actually do
+ // merging with a merge buffer, otherwise the query will allocate too many merge buffers. This is potentially
+ // sub-optimal as it will involve materializing the results for each appendable segment before starting to feed
+ // them into the outer merge buffer. I'm not sure of a better way to do this without tweaking how realtime servers
+ // do queries.
final boolean forceChainedExecution = query.context().getBoolean(
CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION,
false
diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java
index 0013c4f84fcd..f3e32cfb2e09 100644
--- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java
@@ -40,7 +40,7 @@
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.ResourceLimitExceededException;
import org.apache.druid.query.SegmentDescriptor;
-import org.apache.druid.query.SinkQueryRunners;
+import org.apache.druid.query.AppendableSegmentQueryRunners;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.spec.MultipleSpecificSegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
@@ -142,9 +142,9 @@ public QueryRunner mergeRunners(
for (int i = 0; i < queryRunnersOrdered.size(); i++) {
intervalsAndRunnersOrdered.add(new Pair<>(intervalsOrdered.get(i), queryRunnersOrdered.get(i)));
}
- } else if (queryRunners instanceof SinkQueryRunners) {
- ((SinkQueryRunners) queryRunners).runnerIntervalMappingIterator()
- .forEachRemaining(intervalsAndRunnersOrdered::add);
+ } else if (queryRunners instanceof AppendableSegmentQueryRunners) {
+ ((AppendableSegmentQueryRunners) queryRunners).runnerIntervalMappingIterator()
+ .forEachRemaining(intervalsAndRunnersOrdered::add);
} else {
throw new ISE("Number of segment descriptors does not equal number of "
+ "query runners...something went wrong!");
diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java
index 98588de1d534..842c9de4fa40 100644
--- a/processing/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java
+++ b/processing/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java
@@ -98,8 +98,8 @@ public HashPartitionFunction getPartitionFunction()
@Override
public PartitionChunk createChunk(T obj)
{
- // This method can be called in AppenderatorImpl to create a sinkTimeline.
- // The sinkTimeline doesn't seem in use in batch ingestion, let's set 'chunks' to 0 for now.
+ // This method can be called in AppenderatorImpl to create a appendableSegmentTimeline.
+ // The appendableSegmentTimeline doesn't seem in use in batch ingestion, let's set 'chunks' to 0 for now.
// HashBasedNumberedShardSpec is using NumberedPartitionChunk, so we use it here too.
return new NumberedPartitionChunk<>(partitionId, 0, obj);
}
diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java
index 52b3069f0b4e..df456db15c3a 100644
--- a/processing/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java
+++ b/processing/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java
@@ -61,8 +61,8 @@ public NumberedShardSpec convert(int numTotalPartitions)
@Override
public PartitionChunk createChunk(T obj)
{
- // This method can be called in AppenderatorImpl to create a sinkTimeline.
- // The sinkTimeline doesn't seem in use in batch ingestion, let's set 'chunks' to 0 for now.
+ // This method can be called in AppenderatorImpl to create a appendableSegmentTimeline.
+ // The appendableSegmentTimeline doesn't seem in use in batch ingestion, let's set 'chunks' to 0 for now.
return new NumberedPartitionChunk<>(partitionId, 0, obj);
}
diff --git a/publications/radstack/radstack.tex b/publications/radstack/radstack.tex
index ae730e0f2538..7dd499cb8b34 100644
--- a/publications/radstack/radstack.tex
+++ b/publications/radstack/radstack.tex
@@ -516,7 +516,7 @@ \subsection{Stream Processing}
but have since switched to Samza). Samza provides an API to write jobs that run
over a sequence of tuples and perform operations over those tuples in a
user-defined way. The input to each job is provided by Kafka, which can also
-act as a sink for the output of the job. Samza jobs are executed in a resource
+act as a appendableSegment for the output of the job. Samza jobs are executed in a resource
management and task execution framework such as
YARN\cite{vavilapalli2013apache}. It is beyond the scope of this paper to go
into the full details of Kafka/YARN/Samza interactions, but more information is
diff --git a/server/src/main/java/org/apache/druid/client/cache/CachePopulator.java b/server/src/main/java/org/apache/druid/client/cache/CachePopulator.java
index dbbade1cf6a0..2d4a2a81236a 100644
--- a/server/src/main/java/org/apache/druid/client/cache/CachePopulator.java
+++ b/server/src/main/java/org/apache/druid/client/cache/CachePopulator.java
@@ -20,6 +20,7 @@
package org.apache.druid.client.cache;
import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.segment.realtime.appenderator.AppendableSegmentQuerySegmentWalker;
import java.util.function.Function;
@@ -34,7 +35,7 @@
* historicals: {@link org.apache.druid.server.coordination.ServerManager} and
* {@link org.apache.druid.client.CachingQueryRunner}
*
- * realtime tasks: {@link org.apache.druid.segment.realtime.appenderator.SinkQuerySegmentWalker} and
+ * realtime tasks: {@link AppendableSegmentQuerySegmentWalker} and
* {@link org.apache.druid.client.CachingQueryRunner}
*
* brokers: {@link org.apache.druid.server.ClientQuerySegmentWalker} and
diff --git a/server/src/main/java/org/apache/druid/guice/FirehoseModule.java b/server/src/main/java/org/apache/druid/guice/FirehoseModule.java
deleted file mode 100644
index fe6461bf2158..000000000000
--- a/server/src/main/java/org/apache/druid/guice/FirehoseModule.java
+++ /dev/null
@@ -1,55 +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.guice;
-
-import com.fasterxml.jackson.databind.Module;
-import com.fasterxml.jackson.databind.jsontype.NamedType;
-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.EventReceiverFirehoseFactory;
-import org.apache.druid.segment.realtime.firehose.FixedCountFirehoseFactory;
-import org.apache.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory;
-
-import java.util.Collections;
-import java.util.List;
-
-public class FirehoseModule implements DruidModule
-{
- @Override
- public void configure(Binder binder)
- {
- }
-
- @Override
- public List extends Module> getJacksonModules()
- {
- return Collections.singletonList(
- new SimpleModule("FirehoseModule")
- .registerSubtypes(
- new NamedType(ClippedFirehoseFactory.class, "clipped"),
- new NamedType(TimedShutoffFirehoseFactory.class, "timed"),
- new NamedType(EventReceiverFirehoseFactory.class, "receiver"),
- new NamedType(FixedCountFirehoseFactory.class, "fixedCount")
- )
- );
- }
-}
diff --git a/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java b/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java
index 59516f8d4c49..e07ac5ed1155 100644
--- a/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java
+++ b/server/src/main/java/org/apache/druid/initialization/CoreInjectorBuilder.java
@@ -29,7 +29,6 @@
import org.apache.druid.guice.DruidSecondaryModule;
import org.apache.druid.guice.ExpressionModule;
import org.apache.druid.guice.ExtensionsModule;
-import org.apache.druid.guice.FirehoseModule;
import org.apache.druid.guice.JacksonConfigManagerModule;
import org.apache.druid.guice.JavaScriptModule;
import org.apache.druid.guice.LifecycleModule;
@@ -123,7 +122,6 @@ public CoreInjectorBuilder forServer()
new CoordinatorDiscoveryModule(),
new LocalDataStorageDruidModule(),
new TombstoneDataStorageModule(),
- new FirehoseModule(),
new JavaScriptModule(),
new AuthenticatorModule(),
new AuthenticatorMapperModule(),
diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskRetryPolicy.java b/server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskRetryPolicy.java
index 36d84f2dc4e7..057201b8564c 100644
--- a/server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskRetryPolicy.java
+++ b/server/src/main/java/org/apache/druid/rpc/indexing/SpecificTaskRetryPolicy.java
@@ -23,7 +23,7 @@
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.rpc.ServiceRetryPolicy;
import org.apache.druid.rpc.StandardRetryPolicy;
-import org.apache.druid.segment.realtime.firehose.ChatHandlerResource;
+import org.apache.druid.segment.realtime.ChatHandlerResource;
import org.jboss.netty.handler.codec.http.HttpResponse;
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/AppendableSegment.java
similarity index 73%
rename from server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java
rename to server/src/main/java/org/apache/druid/segment/realtime/AppendableSegment.java
index 44d0db3cb7f6..51bd7d89aef8 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/AppendableSegment.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.segment.realtime.sink;
+package org.apache.druid.segment.realtime;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Predicate;
@@ -47,7 +47,6 @@
import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter;
import org.apache.druid.segment.incremental.IndexSizeExceededException;
import org.apache.druid.segment.indexing.DataSchema;
-import org.apache.druid.segment.realtime.FireHydrant;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.Overshadowable;
import org.apache.druid.timeline.partition.ShardSpec;
@@ -70,15 +69,15 @@
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
-public class Sink implements Iterable, Overshadowable
+public class AppendableSegment implements Iterable, Overshadowable
{
private static final IncrementalIndexAddResult NOT_WRITABLE = new IncrementalIndexAddResult(-1, -1, "not writable");
private static final IncrementalIndexAddResult ALREADY_SWAPPED =
new IncrementalIndexAddResult(-1, -1, "write after index swapped");
- private static final Logger log = new Logger(Sink.class);
+ private static final Logger log = new Logger(AppendableSegment.class);
- private final Object hydrantLock = new Object();
+ private final Object partialSegmentLock = new Object();
private final Interval interval;
private final DataSchema schema;
private final ShardSpec shardSpec;
@@ -87,11 +86,11 @@ public class Sink implements Iterable, Overshadowable
private final int maxRowsInMemory;
private final long maxBytesInMemory;
private final boolean useMaxMemoryEstimates;
- private final CopyOnWriteArrayList hydrants = new CopyOnWriteArrayList<>();
+ private final CopyOnWriteArrayList partialSegments = new CopyOnWriteArrayList<>();
private final LinkedHashSet dimOrder = new LinkedHashSet<>();
- // columns excluding current index (the in-memory fire hydrant), includes __time column
+ // columns excluding current index (the in-memory PartialSegment), includes __time column
private final LinkedHashSet columnsExcludingCurrIndex = new LinkedHashSet<>();
// column types for columns in {@code columnsExcludingCurrIndex}
@@ -99,10 +98,10 @@ public class Sink implements Iterable, Overshadowable
private final AtomicInteger numRowsExcludingCurrIndex = new AtomicInteger();
- private volatile FireHydrant currHydrant;
+ private volatile PartialSegment currentPartialSegment;
private volatile boolean writable = true;
- public Sink(
+ public AppendableSegment(
Interval interval,
DataSchema schema,
ShardSpec shardSpec,
@@ -126,7 +125,7 @@ public Sink(
);
}
- public Sink(
+ public AppendableSegment(
Interval interval,
DataSchema schema,
ShardSpec shardSpec,
@@ -135,7 +134,7 @@ public Sink(
int maxRowsInMemory,
long maxBytesInMemory,
boolean useMaxMemoryEstimates,
- List hydrants
+ List partialSegments
)
{
this.schema = schema;
@@ -148,13 +147,13 @@ public Sink(
this.useMaxMemoryEstimates = useMaxMemoryEstimates;
int maxCount = -1;
- for (int i = 0; i < hydrants.size(); ++i) {
- final FireHydrant hydrant = hydrants.get(i);
- if (hydrant.getCount() <= maxCount) {
- throw new ISE("hydrant[%s] not the right count[%s]", hydrant, i);
+ for (int i = 0; i < partialSegments.size(); ++i) {
+ final PartialSegment partialSegment = partialSegments.get(i);
+ if (partialSegment.getCount() <= maxCount) {
+ throw new ISE("partialSegment[%s] not the right count[%s]", partialSegment, i);
}
- maxCount = hydrant.getCount();
- ReferenceCountingSegment segment = hydrant.getIncrementedSegment();
+ maxCount = partialSegment.getCount();
+ ReferenceCountingSegment segment = partialSegment.getIncrementedSegment();
try {
QueryableIndex index = segment.asQueryableIndex();
overwriteIndexDimensions(new QueryableIndexStorageAdapter(index));
@@ -164,7 +163,7 @@ public Sink(
segment.decrement();
}
}
- this.hydrants.addAll(hydrants);
+ this.partialSegments.addAll(partialSegments);
makeNewCurrIndex(interval.getStartMillis(), schema);
}
@@ -174,25 +173,25 @@ public Interval getInterval()
return interval;
}
- public FireHydrant getCurrHydrant()
+ public PartialSegment getCurrentPartialSegment()
{
- return currHydrant;
+ return currentPartialSegment;
}
public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCheck) throws IndexSizeExceededException
{
- if (currHydrant == null) {
- throw new IAE("No currHydrant but given row[%s]", row);
+ if (currentPartialSegment == null) {
+ throw new IAE("No currentPartialSegment but given row[%s]", row);
}
- synchronized (hydrantLock) {
+ synchronized (partialSegmentLock) {
if (!writable) {
return NOT_WRITABLE;
}
- IncrementalIndex index = currHydrant.getIndex();
+ IncrementalIndex index = currentPartialSegment.getIndex();
if (index == null) {
- return ALREADY_SWAPPED; // the hydrant was swapped without being replaced
+ return ALREADY_SWAPPED; // the partialSegment was swapped without being replaced
}
return index.add(row, skipMaxRowsInMemoryCheck);
@@ -201,15 +200,15 @@ public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCh
public boolean canAppendRow()
{
- synchronized (hydrantLock) {
- return writable && currHydrant != null && currHydrant.getIndex().canAppendRow();
+ synchronized (partialSegmentLock) {
+ return writable && currentPartialSegment != null && currentPartialSegment.getIndex().canAppendRow();
}
}
public boolean isEmpty()
{
- synchronized (hydrantLock) {
- return hydrants.size() == 1 && currHydrant.getIndex().isEmpty();
+ synchronized (partialSegmentLock) {
+ return partialSegments.size() == 1 && currentPartialSegment.getIndex().isEmpty();
}
}
@@ -219,19 +218,19 @@ public boolean isWritable()
}
/**
- * If currHydrant is A, creates a new index B, sets currHydrant to B and returns A.
+ * If currPartialSegment is A, creates a new index B, sets currPartialSegment to B and returns A.
*
* @return the current index after swapping in a new one
*/
- public FireHydrant swap()
+ public PartialSegment swap()
{
return makeNewCurrIndex(interval.getStartMillis(), schema);
}
public boolean swappable()
{
- synchronized (hydrantLock) {
- return writable && currHydrant.getIndex() != null && currHydrant.getIndex().size() != 0;
+ synchronized (partialSegmentLock) {
+ return writable && currentPartialSegment.getIndex() != null && currentPartialSegment.getIndex().size() != 0;
}
}
@@ -247,7 +246,7 @@ public boolean finished()
*/
public boolean finishWriting()
{
- synchronized (hydrantLock) {
+ synchronized (partialSegmentLock) {
if (!writable) {
return false;
}
@@ -273,52 +272,52 @@ public DataSegment getSegment()
public int getNumRows()
{
- synchronized (hydrantLock) {
+ synchronized (partialSegmentLock) {
return numRowsExcludingCurrIndex.get() + getNumRowsInMemory();
}
}
public int getNumRowsInMemory()
{
- synchronized (hydrantLock) {
- IncrementalIndex index = currHydrant.getIndex();
+ synchronized (partialSegmentLock) {
+ IncrementalIndex index = currentPartialSegment.getIndex();
if (index == null) {
return 0;
}
- return currHydrant.getIndex().size();
+ return currentPartialSegment.getIndex().size();
}
}
public long getBytesInMemory()
{
- synchronized (hydrantLock) {
- IncrementalIndex index = currHydrant.getIndex();
+ synchronized (partialSegmentLock) {
+ IncrementalIndex index = currentPartialSegment.getIndex();
if (index == null) {
return 0;
}
- return currHydrant.getIndex().getBytesInMemory().get();
+ return currentPartialSegment.getIndex().getBytesInMemory().get();
}
}
/**
- * Acquire references to all {@link FireHydrant} that represent this sink. Returns null if they cannot all be
+ * Acquire references to all {@link PartialSegment} that represent this sink. Returns null if they cannot all be
* acquired, possibly because they were closed (swapped to null) concurrently with this method being called.
*
* @param segmentMapFn from {@link org.apache.druid.query.DataSource#createSegmentMapFunction(Query, AtomicLong)}
* @param skipIncrementalSegment whether in-memory {@link IncrementalIndex} segments should be skipped
*/
@Nullable
- public List acquireSegmentReferences(
+ public List acquireSegmentReferences(
final Function segmentMapFn,
final boolean skipIncrementalSegment
)
{
- return acquireSegmentReferences(hydrants, segmentMapFn, skipIncrementalSegment);
+ return acquireSegmentReferences(partialSegments, segmentMapFn, skipIncrementalSegment);
}
- private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema)
+ private PartialSegment makeNewCurrIndex(long minTimestamp, DataSchema schema)
{
final IncrementalIndexSchema indexSchema = new IncrementalIndexSchema.Builder()
.withMinTimestamp(minTimestamp)
@@ -338,22 +337,22 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema)
.setUseMaxMemoryEstimates(useMaxMemoryEstimates)
.build();
- final FireHydrant old;
- synchronized (hydrantLock) {
+ final PartialSegment old;
+ synchronized (partialSegmentLock) {
if (writable) {
- old = currHydrant;
+ old = currentPartialSegment;
int newCount = 0;
- int numHydrants = hydrants.size();
- if (numHydrants > 0) {
- FireHydrant lastHydrant = hydrants.get(numHydrants - 1);
+ int numPartialSEgments = partialSegments.size();
+ if (numPartialSEgments > 0) {
+ PartialSegment lastParialSegment = partialSegments.get(numPartialSEgments - 1);
Map oldFormat = null;
- newCount = lastHydrant.getCount() + 1;
+ newCount = lastParialSegment.getCount() + 1;
boolean customDimensions = !indexSchema.getDimensionsSpec().hasCustomDimensions();
- if (lastHydrant.hasSwapped()) {
+ if (lastParialSegment.hasSwapped()) {
oldFormat = new HashMap<>();
- ReferenceCountingSegment segment = lastHydrant.getIncrementedSegment();
+ ReferenceCountingSegment segment = lastParialSegment.getIncrementedSegment();
try {
QueryableIndex oldIndex = segment.asQueryableIndex();
overwriteIndexDimensions(new QueryableIndexStorageAdapter(oldIndex));
@@ -368,7 +367,7 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema)
segment.decrement();
}
} else {
- IncrementalIndex oldIndex = lastHydrant.getIndex();
+ IncrementalIndex oldIndex = lastParialSegment.getIndex();
overwriteIndexDimensions(new IncrementalIndexStorageAdapter(oldIndex));
if (customDimensions) {
dimOrder.addAll(oldIndex.getDimensionOrder());
@@ -379,11 +378,11 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema)
newIndex.loadDimensionIterable(dimOrder, oldFormat);
}
}
- currHydrant = new FireHydrant(newIndex, newCount, getSegment().getId());
+ currentPartialSegment = new PartialSegment(newIndex, newCount, getSegment().getId());
if (old != null) {
numRowsExcludingCurrIndex.addAndGet(old.getIndex().size());
}
- hydrants.add(currHydrant);
+ partialSegments.add(currentPartialSegment);
} else {
// Oops, someone called finishWriting while we were making this new index.
newIndex.close();
@@ -407,11 +406,11 @@ private void overwriteIndexDimensions(StorageAdapter storageAdapter)
}
/**
- * Get column information from all the {@link FireHydrant}'s.
+ * Get column information from all the {@link PartialSegment}'s.
*/
public RowSignature getSignature()
{
- synchronized (hydrantLock) {
+ synchronized (partialSegmentLock) {
RowSignature.Builder builder = RowSignature.builder();
builder.addTimeColumn();
@@ -422,7 +421,8 @@ public RowSignature getSignature()
}
}
- IncrementalIndexStorageAdapter incrementalIndexStorageAdapter = new IncrementalIndexStorageAdapter(currHydrant.getIndex());
+ IncrementalIndexStorageAdapter incrementalIndexStorageAdapter = new IncrementalIndexStorageAdapter(
+ currentPartialSegment.getIndex());
RowSignature incrementalIndexSignature = incrementalIndexStorageAdapter.getRowSignature();
for (String dim : incrementalIndexSignature.getColumnNames()) {
@@ -440,14 +440,14 @@ public RowSignature getSignature()
}
@Override
- public Iterator iterator()
+ public Iterator iterator()
{
return Iterators.filter(
- hydrants.iterator(),
- new Predicate()
+ partialSegments.iterator(),
+ new Predicate()
{
@Override
- public boolean apply(FireHydrant input)
+ public boolean apply(PartialSegment input)
{
final IncrementalIndex index = input.getIndex();
return index == null || index.size() != 0;
@@ -466,10 +466,10 @@ public String toString()
}
@Override
- public boolean overshadows(Sink other)
+ public boolean overshadows(AppendableSegment other)
{
// Sink is currently used in timeline only for querying stream data.
- // In this case, sinks never overshadow each other.
+ // In this case, AppendableSegment never overshadow each other.
return false;
}
@@ -508,31 +508,31 @@ public short getAtomicUpdateGroupSize()
* method instead of testing {@link #acquireSegmentReferences(Function, boolean)} directly).
*/
@VisibleForTesting
- static List acquireSegmentReferences(
- final List hydrants,
+ static List acquireSegmentReferences(
+ final List partialSegments,
final Function segmentMapFn,
final boolean skipIncrementalSegment
)
{
- final List retVal = new ArrayList<>(hydrants.size());
+ final List retVal = new ArrayList<>(partialSegments.size());
try {
- for (final FireHydrant hydrant : hydrants) {
- // Hydrant might swap at any point, but if it's swapped at the start
+ for (final PartialSegment partialSegment : partialSegments) {
+ // PartialSegment might swap at any point, but if it's swapped at the start
// then we know it's *definitely* swapped.
- final boolean hydrantDefinitelySwapped = hydrant.hasSwapped();
+ final boolean partialSegmentDefinitelySwapped = partialSegment.hasSwapped();
- if (skipIncrementalSegment && !hydrantDefinitelySwapped) {
+ if (skipIncrementalSegment && !partialSegmentDefinitelySwapped) {
continue;
}
- final Optional> maybeHolder = hydrant.getSegmentForQuery(segmentMapFn);
+ final Optional> maybeHolder = partialSegment.getSegmentForQuery(segmentMapFn);
if (maybeHolder.isPresent()) {
final Pair holder = maybeHolder.get();
- retVal.add(new SinkSegmentReference(hydrant.getCount(), holder.lhs, hydrantDefinitelySwapped, holder.rhs));
+ retVal.add(new PartialSegmentReference(partialSegment.getCount(), holder.lhs, partialSegmentDefinitelySwapped, holder.rhs));
} else {
- // Cannot acquire this hydrant. Release all others previously acquired and return null.
- for (final SinkSegmentReference reference : retVal) {
+ // Cannot acquire this partialSegment. Release all others previously acquired and return null.
+ for (final PartialSegmentReference reference : retVal) {
reference.close();
}
@@ -544,7 +544,7 @@ static List acquireSegmentReferences(
}
catch (Throwable e) {
// Release all references previously acquired and throw the error.
- for (final SinkSegmentReference reference : retVal) {
+ for (final PartialSegmentReference reference : retVal) {
CloseableUtils.closeAndSuppressExceptions(reference, e::addSuppressed);
}
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandler.java b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandler.java
similarity index 95%
rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandler.java
rename to server/src/main/java/org/apache/druid/segment/realtime/ChatHandler.java
index 004b6c04d480..56f8330fdcbb 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandler.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandler.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.segment.realtime.firehose;
+package org.apache.druid.segment.realtime;
/**
* Objects that can be registered with a {@link ServiceAnnouncingChatHandlerProvider} and provide http endpoints for indexing-related
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerProvider.java b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandlerProvider.java
similarity index 97%
rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerProvider.java
rename to server/src/main/java/org/apache/druid/segment/realtime/ChatHandlerProvider.java
index 3af0d5c37f00..f19e25f37561 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerProvider.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandlerProvider.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.segment.realtime.firehose;
+package org.apache.druid.segment.realtime;
import com.google.common.base.Optional;
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResource.java b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandlerResource.java
similarity index 98%
rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResource.java
rename to server/src/main/java/org/apache/druid/segment/realtime/ChatHandlerResource.java
index 1cd579d27340..4423a0d0f0fa 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResource.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandlerResource.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.segment.realtime.firehose;
+package org.apache.druid.segment.realtime;
import com.google.common.base.Optional;
import com.google.common.collect.Iterables;
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlers.java b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandlers.java
similarity index 97%
rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlers.java
rename to server/src/main/java/org/apache/druid/segment/realtime/ChatHandlers.java
index 4f6e99217d51..4d971db81e08 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlers.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/ChatHandlers.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.segment.realtime.firehose;
+package org.apache.druid.segment.realtime;
import org.apache.druid.server.security.Access;
import org.apache.druid.server.security.Action;
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/sink/Committers.java b/server/src/main/java/org/apache/druid/segment/realtime/Committers.java
similarity index 96%
rename from server/src/main/java/org/apache/druid/segment/realtime/sink/Committers.java
rename to server/src/main/java/org/apache/druid/segment/realtime/Committers.java
index 8efd117b06cf..178693f0e57c 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/sink/Committers.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/Committers.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.segment.realtime.sink;
+package org.apache.druid.segment.realtime;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/NoopChatHandlerProvider.java b/server/src/main/java/org/apache/druid/segment/realtime/NoopChatHandlerProvider.java
similarity index 96%
rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/NoopChatHandlerProvider.java
rename to server/src/main/java/org/apache/druid/segment/realtime/NoopChatHandlerProvider.java
index 5c49b9ac292e..9480bdf2bc7e 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/NoopChatHandlerProvider.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/NoopChatHandlerProvider.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.segment.realtime.firehose;
+package org.apache.druid.segment.realtime;
import com.google.common.base.Optional;
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java b/server/src/main/java/org/apache/druid/segment/realtime/PartialSegment.java
similarity index 86%
rename from server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java
rename to server/src/main/java/org/apache/druid/segment/realtime/PartialSegment.java
index 5f1a88f2ea97..9f740c6e895e 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/PartialSegment.java
@@ -39,14 +39,15 @@
import java.util.function.Function;
/**
+ * Intermediary segment of {@link AppendableSegment} which will be combined with others when merging during publish.
*/
-public class FireHydrant
+public class PartialSegment
{
private final int count;
private final AtomicReference adapter;
private volatile IncrementalIndex index;
- public FireHydrant(IncrementalIndex index, int count, SegmentId segmentId)
+ public PartialSegment(IncrementalIndex index, int count, SegmentId segmentId)
{
this.index = index;
this.adapter = new AtomicReference<>(
@@ -55,7 +56,7 @@ public FireHydrant(IncrementalIndex index, int count, SegmentId segmentId)
this.count = count;
}
- public FireHydrant(Segment adapter, int count)
+ public PartialSegment(Segment adapter, int count)
{
this.index = null;
this.adapter = new AtomicReference<>(ReferenceCountingSegment.wrapRootGenerationSegment(adapter));
@@ -149,10 +150,10 @@ public ReferenceCountingSegment getIncrementedSegment()
// swap, the new segment should already be visible.
ReferenceCountingSegment newSegment = adapter.get();
if (segment == newSegment) {
- throw new ISE("segment.close() is called somewhere outside FireHydrant.swapSegment()");
+ throw new ISE("segment.close() is called somewhere outside PartialSegment.swapSegment()");
}
if (newSegment == null) {
- throw new ISE("FireHydrant was 'closed' by swapping segment to null while acquiring a segment");
+ throw new ISE("PartialSegment was 'closed' by swapping segment to null while acquiring a segment");
}
segment = newSegment;
// Spin loop.
@@ -168,20 +169,20 @@ public Pair getAndIncrementSegment()
/**
* This method is like a combined form of {@link #getIncrementedSegment} and {@link #getAndIncrementSegment} that
* deals in {@link SegmentReference} instead of directly with {@link ReferenceCountingSegment} in order to acquire
- * reference count for both hydrant's segment and any tracked joinables taking part in the query.
+ * reference count for both PartialSegment and any tracked joinables taking part in the query.
*/
public Optional> getSegmentForQuery(
Function segmentMapFn
)
{
- ReferenceCountingSegment sinkSegment = adapter.get();
+ ReferenceCountingSegment appendableSegment = adapter.get();
- if (sinkSegment == null) {
+ if (appendableSegment == null) {
// adapter can be null if this segment is removed (swapped to null) while being queried.
return Optional.empty();
}
- SegmentReference segment = segmentMapFn.apply(sinkSegment);
+ SegmentReference segment = segmentMapFn.apply(appendableSegment);
while (true) {
Optional reference = segment.acquireReferences();
if (reference.isPresent()) {
@@ -190,26 +191,26 @@ public Optional> getSegmentForQuery(
}
// segment.acquireReferences() returned false, means it is closed. Since close() in swapSegment() happens after
// segment swap, the new segment should already be visible.
- ReferenceCountingSegment newSinkSegment = adapter.get();
- if (newSinkSegment == null) {
+ ReferenceCountingSegment newAppendableSegment = adapter.get();
+ if (newAppendableSegment == null) {
// adapter can be null if this segment is removed (swapped to null) while being queried.
return Optional.empty();
}
- if (sinkSegment == newSinkSegment) {
- if (newSinkSegment.isClosed()) {
- throw new ISE("segment.close() is called somewhere outside FireHydrant.swapSegment()");
+ if (appendableSegment == newAppendableSegment) {
+ if (newAppendableSegment.isClosed()) {
+ throw new ISE("segment.close() is called somewhere outside PartialSegment.swapSegment()");
}
// if segment is not closed, but is same segment it means we are having trouble getting references for joinables
// of a HashJoinSegment created by segmentMapFn
return Optional.empty();
}
- segment = segmentMapFn.apply(newSinkSegment);
+ segment = segmentMapFn.apply(newAppendableSegment);
// Spin loop.
}
}
@VisibleForTesting
- public ReferenceCountingSegment getHydrantSegment()
+ public ReferenceCountingSegment getPartialSegment()
{
return adapter.get();
}
@@ -219,7 +220,7 @@ public String toString()
{
// Do not include IncrementalIndex in toString as AbstractIndex.toString() actually prints
// all the rows in the index
- return "FireHydrant{" +
+ return "PartialSegment{" +
"queryable=" + (adapter.get() == null ? "null" : adapter.get().getId()) +
", count=" + count +
'}';
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/sink/SinkSegmentReference.java b/server/src/main/java/org/apache/druid/segment/realtime/PartialSegmentReference.java
similarity index 68%
rename from server/src/main/java/org/apache/druid/segment/realtime/sink/SinkSegmentReference.java
rename to server/src/main/java/org/apache/druid/segment/realtime/PartialSegmentReference.java
index c24cf1186614..dc72ecbd2c75 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/sink/SinkSegmentReference.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/PartialSegmentReference.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.segment.realtime.sink;
+package org.apache.druid.segment.realtime;
import org.apache.druid.segment.SegmentReference;
@@ -27,31 +27,37 @@
import java.util.function.Function;
/**
- * Segment reference returned by {@link Sink#acquireSegmentReferences(Function, boolean)}. Must be closed in order
- * to release the reference.
+ * Segment reference for a {@link PartialSegment} returned by
+ * {@link AppendableSegment#acquireSegmentReferences(Function, boolean)}. Must be closed in order to release the
+ * reference.
*/
-public class SinkSegmentReference implements Closeable
+public class PartialSegmentReference implements Closeable
{
- private final int hydrantNumber;
+ private final int partialSegmentNumber;
private final SegmentReference segment;
private final boolean immutable;
private final Closeable releaser;
- public SinkSegmentReference(int hydrantNumber, SegmentReference segment, boolean immutable, Closeable releaser)
+ public PartialSegmentReference(
+ int partialSegmentNumber,
+ SegmentReference segment,
+ boolean immutable,
+ Closeable releaser
+ )
{
- this.hydrantNumber = hydrantNumber;
+ this.partialSegmentNumber = partialSegmentNumber;
this.segment = segment;
this.immutable = immutable;
this.releaser = releaser;
}
/**
- * Index of the {@link org.apache.druid.segment.realtime.FireHydrant} within the {@link Sink} that this segment
+ * Index of the {@link PartialSegment} within the {@link AppendableSegment} that this segment
* reference came from.
*/
- public int getHydrantNumber()
+ public int getPartialSegmentNumber()
{
- return hydrantNumber;
+ return partialSegmentNumber;
}
/**
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/SegmentGenerationMetrics.java b/server/src/main/java/org/apache/druid/segment/realtime/SegmentGenerationMetrics.java
index 564a3322bf93..cf48b2f91065 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/SegmentGenerationMetrics.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/SegmentGenerationMetrics.java
@@ -48,7 +48,7 @@ public class SegmentGenerationMetrics
private final AtomicLong mergeCpuTime = new AtomicLong(0);
private final AtomicLong persistCpuTime = new AtomicLong(0);
private final AtomicLong handOffCount = new AtomicLong(0);
- private final AtomicLong sinkCount = new AtomicLong(0);
+ private final AtomicLong appendableSegmentCount = new AtomicLong(0);
private final AtomicLong messageMaxTimestamp = new AtomicLong(0);
private final AtomicLong messageGap = new AtomicLong(0);
private final AtomicBoolean processingDone = new AtomicBoolean(false);
@@ -100,9 +100,9 @@ public void incrementHandOffCount()
handOffCount.incrementAndGet();
}
- public void setSinkCount(long sinkCount)
+ public void setAppendableSegmentCount(long appendableSegmentCount)
{
- this.sinkCount.set(sinkCount);
+ this.appendableSegmentCount.set(appendableSegmentCount);
}
public void reportMessageMaxTimestamp(long messageMaxTimestamp)
@@ -190,9 +190,9 @@ public long handOffCount()
return handOffCount.get();
}
- public long sinkCount()
+ public long appendableSegmentCount()
{
- return sinkCount.get();
+ return appendableSegmentCount.get();
}
public long messageGap()
@@ -219,7 +219,7 @@ public SegmentGenerationMetrics snapshot()
retVal.mergeCpuTime.set(mergeCpuTime.get());
retVal.persistCpuTime.set(persistCpuTime.get());
retVal.handOffCount.set(handOffCount.get());
- retVal.sinkCount.set(sinkCount.get());
+ retVal.appendableSegmentCount.set(appendableSegmentCount.get());
retVal.messageMaxTimestamp.set(messageMaxTimestamp.get());
retVal.maxSegmentHandoffTime.set(maxSegmentHandoffTime.get());
retVal.mergedRows.set(mergedRows.get());
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProvider.java b/server/src/main/java/org/apache/druid/segment/realtime/ServiceAnnouncingChatHandlerProvider.java
similarity index 98%
rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProvider.java
rename to server/src/main/java/org/apache/druid/segment/realtime/ServiceAnnouncingChatHandlerProvider.java
index 802be54cc115..da3975e4545a 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProvider.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/ServiceAnnouncingChatHandlerProvider.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.segment.realtime.firehose;
+package org.apache.druid.segment.realtime;
import com.google.common.base.Optional;
import com.google.inject.Inject;
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/WindowedStorageAdapter.java b/server/src/main/java/org/apache/druid/segment/realtime/WindowedStorageAdapter.java
similarity index 96%
rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/WindowedStorageAdapter.java
rename to server/src/main/java/org/apache/druid/segment/realtime/WindowedStorageAdapter.java
index 26d0b41013da..9b4bd1e4ea7f 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/WindowedStorageAdapter.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/WindowedStorageAdapter.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.segment.realtime.firehose;
+package org.apache.druid.segment.realtime;
import org.apache.druid.segment.StorageAdapter;
import org.joda.time.Interval;
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppendableSegmentQuerySegmentWalker.java
similarity index 79%
rename from server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java
rename to server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppendableSegmentQuerySegmentWalker.java
index 72cfe4de2615..0d9c00d9ab00 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppendableSegmentQuerySegmentWalker.java
@@ -52,15 +52,15 @@
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.ReportTimelineMissingSegmentQueryRunner;
import org.apache.druid.query.SegmentDescriptor;
-import org.apache.druid.query.SinkQueryRunners;
+import org.apache.druid.query.AppendableSegmentQueryRunners;
import org.apache.druid.query.planning.DataSourceAnalysis;
import org.apache.druid.query.spec.SpecificSegmentQueryRunner;
import org.apache.druid.query.spec.SpecificSegmentSpec;
import org.apache.druid.segment.SegmentReference;
import org.apache.druid.segment.StorageAdapter;
-import org.apache.druid.segment.realtime.FireHydrant;
-import org.apache.druid.segment.realtime.sink.Sink;
-import org.apache.druid.segment.realtime.sink.SinkSegmentReference;
+import org.apache.druid.segment.realtime.AppendableSegment;
+import org.apache.druid.segment.realtime.PartialSegment;
+import org.apache.druid.segment.realtime.PartialSegmentReference;
import org.apache.druid.server.ResourceIdPopulatingQueryRunner;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.VersionedIntervalTimeline;
@@ -84,14 +84,14 @@
/**
* Query handler for indexing tasks.
*/
-public class SinkQuerySegmentWalker implements QuerySegmentWalker
+public class AppendableSegmentQuerySegmentWalker implements QuerySegmentWalker
{
- private static final EmittingLogger log = new EmittingLogger(SinkQuerySegmentWalker.class);
+ private static final EmittingLogger log = new EmittingLogger(AppendableSegmentQuerySegmentWalker.class);
private static final String CONTEXT_SKIP_INCREMENTAL_SEGMENT = "skipIncrementalSegment";
private final String dataSource;
- private final VersionedIntervalTimeline sinkTimeline;
+ private final VersionedIntervalTimeline appendableSegmentTimeline;
private final ObjectMapper objectMapper;
private final ServiceEmitter emitter;
private final QueryRunnerFactoryConglomerate conglomerate;
@@ -102,9 +102,9 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker
private final ConcurrentMap newIdToBasePendingSegment
= new ConcurrentHashMap<>();
- public SinkQuerySegmentWalker(
+ public AppendableSegmentQuerySegmentWalker(
String dataSource,
- VersionedIntervalTimeline sinkTimeline,
+ VersionedIntervalTimeline appendableSegmentTimeline,
ObjectMapper objectMapper,
ServiceEmitter emitter,
QueryRunnerFactoryConglomerate conglomerate,
@@ -115,7 +115,7 @@ public SinkQuerySegmentWalker(
)
{
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
- this.sinkTimeline = Preconditions.checkNotNull(sinkTimeline, "sinkTimeline");
+ this.appendableSegmentTimeline = Preconditions.checkNotNull(appendableSegmentTimeline, "appendableSegmentTimeline");
this.objectMapper = Preconditions.checkNotNull(objectMapper, "objectMapper");
this.emitter = Preconditions.checkNotNull(emitter, "emitter");
this.conglomerate = Preconditions.checkNotNull(conglomerate, "conglomerate");
@@ -134,7 +134,7 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, final
{
final Iterable specs = FunctionalIterable
.create(intervals)
- .transformCat(sinkTimeline::lookup)
+ .transformCat(appendableSegmentTimeline::lookup)
.transformCat(
holder -> FunctionalIterable
.create(holder.getObject())
@@ -187,19 +187,19 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final
// We compute the join cache key here itself so it doesn't need to be re-computed for every segment
final Optional cacheKeyPrefix = Optional.ofNullable(query.getDataSource().getCacheKey());
- // We need to report data for each Sink all-or-nothing, which means we need to acquire references for all
- // subsegments (FireHydrants) of a segment (Sink) at once. To ensure they are properly released even when a
- // query fails or is canceled, we acquire *all* sink reference upfront, and release them all when the main
- // QueryRunner returned by this method is closed. (We can't do the acquisition and releasing at the level of
- // each FireHydrant's runner, since then it wouldn't be properly all-or-nothing on a per-Sink basis.)
- final List allSegmentReferences = new ArrayList<>();
+ // We need to report data for each AppendableSegment all-or-nothing, which means we need to acquire references for
+ // all PartialSegment of an AppendableSegment at once. To ensure they are properly released even when a query fails
+ // or is canceled, we acquire *all* AppendableSegment references upfront, and release them all when the main
+ // QueryRunner returned by this method is closed. (We can't do the acquisition and releasing at the level of each
+ // PartialSegment's runner, since then it wouldn't be properly all-or-nothing on a per-AppendableSegment basis.)
+ final List allSegmentReferences = new ArrayList<>();
final Map segmentIdMap = new HashMap<>();
final LinkedHashMap>> allRunners = new LinkedHashMap<>();
try {
for (final SegmentDescriptor newDescriptor : specs) {
final SegmentDescriptor descriptor = newIdToBasePendingSegment.getOrDefault(newDescriptor, newDescriptor);
- final PartitionChunk chunk = sinkTimeline.findChunk(
+ final PartitionChunk chunk = appendableSegmentTimeline.findChunk(
descriptor.getInterval(),
descriptor.getVersion(),
descriptor.getPartitionNumber()
@@ -213,44 +213,44 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final
continue;
}
- final Sink theSink = chunk.getObject();
- final SegmentId sinkSegmentId = theSink.getSegment().getId();
- segmentIdMap.put(descriptor, sinkSegmentId);
- final List sinkSegmentReferences =
- theSink.acquireSegmentReferences(segmentMapFn, skipIncrementalSegment);
+ final AppendableSegment theAppendableSegment = chunk.getObject();
+ final SegmentId appendableSegmentId = theAppendableSegment.getSegment().getId();
+ segmentIdMap.put(descriptor, appendableSegmentId);
+ final List appendableSegmentReferences =
+ theAppendableSegment.acquireSegmentReferences(segmentMapFn, skipIncrementalSegment);
- if (sinkSegmentReferences == null) {
- // We failed to acquire references for all subsegments. Bail and report the entire sink missing.
+ if (appendableSegmentReferences == null) {
+ // We failed to acquire references for all subsegments. Bail and report the entire appendable segment missing.
allRunners.put(
descriptor,
Collections.singletonList(new ReportTimelineMissingSegmentQueryRunner<>(descriptor))
);
- } else if (sinkSegmentReferences.isEmpty()) {
+ } else if (appendableSegmentReferences.isEmpty()) {
allRunners.put(descriptor, Collections.singletonList(new NoopQueryRunner<>()));
} else {
- allSegmentReferences.addAll(sinkSegmentReferences);
+ allSegmentReferences.addAll(appendableSegmentReferences);
allRunners.put(
descriptor,
- sinkSegmentReferences.stream().map(
+ appendableSegmentReferences.stream().map(
segmentReference -> {
QueryRunner runner = new MetricsEmittingQueryRunner<>(
emitter,
factory.getToolchest(),
factory.createRunner(segmentReference.getSegment()),
QueryMetrics::reportSegmentTime,
- queryMetrics -> queryMetrics.segment(sinkSegmentId.toString())
+ queryMetrics -> queryMetrics.segment(appendableSegmentId.toString())
);
// 1) Only use caching if data is immutable
- // 2) Hydrants are not the same between replicas, make sure cache is local
+ // 2) Partial segments are not the same between replicas, make sure cache is local
if (segmentReference.isImmutable() && cache.isLocal()) {
StorageAdapter storageAdapter = segmentReference.getSegment().asStorageAdapter();
long segmentMinTime = storageAdapter.getMinTime().getMillis();
long segmentMaxTime = storageAdapter.getMaxTime().getMillis();
Interval actualDataInterval = Intervals.utc(segmentMinTime, segmentMaxTime + 1);
runner = new CachingQueryRunner<>(
- makeHydrantCacheIdentifier(sinkSegmentId, segmentReference.getHydrantNumber()),
+ makePartialSegmentCacheIdentifier(appendableSegmentId, segmentReference.getPartialSegmentNumber()),
cacheKeyPrefix,
descriptor,
actualDataInterval,
@@ -275,7 +275,7 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final
factory.getToolchest(),
runner,
QueryMetrics::reportSegmentAndCacheTime,
- queryMetrics -> queryMetrics.segment(sinkSegmentId.toString())
+ queryMetrics -> queryMetrics.segment(appendableSegmentId.toString())
).withWaitMeasuredFromNow();
// Emit CPU time metrics.
@@ -303,9 +303,9 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final
final QueryRunner mergedRunner;
if (query.context().isBySegment()) {
- // bySegment: merge all hydrants for a Sink first, then merge Sinks. Necessary to keep results for the
- // same segment together, but causes additional memory usage due to the extra layer of materialization,
- // so we only do this if we need to.
+ // bySegment: merge all PartialSegment for an AppendableSegment first, then merge AppendableSegment. Necessary
+ // to keep results for the same segment together, but causes additional memory usage due to the extra layer of
+ // materialization, so we only do this if we need to.
mergedRunner = factory.mergeRunners(
queryProcessingPool,
allRunners.entrySet().stream().map(
@@ -320,10 +320,10 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final
).collect(Collectors.toList())
);
} else {
- // Not bySegment: merge all hydrants at the same level, rather than grouped by Sink (segment).
+ // Not bySegment: merge all PartialSegment at the same level, rather than grouped by AppendableSegment.
mergedRunner = factory.mergeRunners(
queryProcessingPool,
- new SinkQueryRunners<>(
+ new AppendableSegmentQueryRunners<>(
allRunners.entrySet().stream().flatMap(
entry ->
entry.getValue().stream().map(
@@ -337,7 +337,7 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final
// 1) Populate resource id to the query
// 2) Merge results using the toolChest, finalize if necessary.
// 3) Measure CPU time of that operation.
- // 4) Release all sink segment references.
+ // 4) Release all appendable segment references.
return new ResourceIdPopulatingQueryRunner<>(
QueryRunnerHelper.makeClosingQueryRunner(
CPUTimeMetricQueryRunner.safeBuild(
@@ -373,21 +373,21 @@ String getDataSource()
return dataSource;
}
- public VersionedIntervalTimeline getSinkTimeline()
+ public VersionedIntervalTimeline getAppendableSegmentTimeline()
{
- return sinkTimeline;
+ return appendableSegmentTimeline;
}
- public static String makeHydrantCacheIdentifier(final FireHydrant hydrant)
+ public static String makePartialSegmentCacheIdentifier(final PartialSegment partialSegment)
{
- return makeHydrantCacheIdentifier(hydrant.getSegmentId(), hydrant.getCount());
+ return makePartialSegmentCacheIdentifier(partialSegment.getSegmentId(), partialSegment.getCount());
}
- public static String makeHydrantCacheIdentifier(final SegmentId segmentId, final int hydrantNumber)
+ public static String makePartialSegmentCacheIdentifier(final SegmentId segmentId, final int partialSegmentNumber)
{
- // Cache ID like segmentId_H0, etc. The 'H' disambiguates subsegment [foo_x_y_z partition 0 hydrant 1]
+ // Cache ID like segmentId_H0, etc. The 'H' disambiguates subsegment [foo_x_y_z partition 0 partialSegment 1]
// from full segment [foo_x_y_z partition 1], and is therefore useful if we ever want the cache to mix full segments
- // with subsegments (hydrants).
- return segmentId + "_H" + hydrantNumber;
+ // with subsegments.
+ return segmentId + "_H" + partialSegmentNumber;
}
}
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkSchemaUtil.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppendableSegmentSchemaUtil.java
similarity index 58%
rename from server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkSchemaUtil.java
rename to server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppendableSegmentSchemaUtil.java
index 9197399313c0..555dff8e2003 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkSchemaUtil.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppendableSegmentSchemaUtil.java
@@ -37,32 +37,33 @@
import java.util.Set;
/**
- * Utility to compute schema for all sinks in a streaming ingestion task, used in {@link StreamAppenderator}.
+ * Utility to compute schema for all {@link org.apache.druid.segment.realtime.AppendableSegment} in a streaming
+ * ingestion task, used in {@link StreamAppenderator}.
*/
-class SinkSchemaUtil
+class AppendableSegmentSchemaUtil
{
/**
- * Compute {@link SegmentSchemas} for the sinks.
+ * Compute {@link SegmentSchemas} for the appendable segments.
*/
@VisibleForTesting
static Optional computeAbsoluteSchema(
- Map> sinkSchemaMap
+ Map> appendableSegmentSchemaMap
)
{
- List sinkSchemas = new ArrayList<>();
+ List appendableSegmentSchemas = new ArrayList<>();
- for (Map.Entry> entry : sinkSchemaMap.entrySet()) {
+ for (Map.Entry> entry : appendableSegmentSchemaMap.entrySet()) {
SegmentId segmentId = entry.getKey();
- RowSignature sinkSignature = entry.getValue().lhs;
+ RowSignature appendableSegmentSignature = entry.getValue().lhs;
List newColumns = new ArrayList<>();
Map columnMapping = new HashMap<>();
- // new Sink
- for (String column : sinkSignature.getColumnNames()) {
+ // new AppendableSegment
+ for (String column : appendableSegmentSignature.getColumnNames()) {
newColumns.add(column);
- sinkSignature.getColumnType(column).ifPresent(type -> columnMapping.put(column, type));
+ appendableSegmentSignature.getColumnType(column).ifPresent(type -> columnMapping.put(column, type));
}
Integer numRows = entry.getValue().rhs;
@@ -77,27 +78,27 @@ static Optional computeAbsoluteSchema(
Collections.emptyList(),
columnMapping
);
- sinkSchemas.add(segmentSchema);
+ appendableSegmentSchemas.add(segmentSchema);
}
}
- return Optional.ofNullable(sinkSchemas.isEmpty() ? null : new SegmentSchemas(sinkSchemas));
+ return Optional.ofNullable(appendableSegmentSchemas.isEmpty() ? null : new SegmentSchemas(appendableSegmentSchemas));
}
/**
- * Compute schema change for the sinks.
+ * Compute schema change for the appendable segments.
*/
@VisibleForTesting
static Optional computeSchemaChange(
- Map> previousSinkSignatureMap,
- Map> currentSinkSignatureMap
+ Map> previousAppendableSegmentSignatureMap,
+ Map> currentAppendableSegmentSignatureMap
)
{
- List sinkSchemas = new ArrayList<>();
+ List appendableSegmentSchemas = new ArrayList<>();
- for (Map.Entry> entry : currentSinkSignatureMap.entrySet()) {
+ for (Map.Entry> entry : currentAppendableSegmentSignatureMap.entrySet()) {
SegmentId segmentId = entry.getKey();
- RowSignature currentSinkSignature = entry.getValue().lhs;
+ RowSignature currentAppendableSegmentSignature = entry.getValue().lhs;
Integer numRows = entry.getValue().rhs;
@@ -110,32 +111,34 @@ static Optional computeSchemaChange(
// if the resultant schema is delta
boolean isDelta = false;
- if (!previousSinkSignatureMap.containsKey(segmentId)) {
- // new Sink
- for (String column : currentSinkSignature.getColumnNames()) {
+ if (!previousAppendableSegmentSignatureMap.containsKey(segmentId)) {
+ // new AppendableSegment
+ for (String column : currentAppendableSegmentSignature.getColumnNames()) {
newColumns.add(column);
- currentSinkSignature.getColumnType(column).ifPresent(type -> currentColumnMapping.put(column, type));
+ currentAppendableSegmentSignature.getColumnType(column).ifPresent(type -> currentColumnMapping.put(column, type));
}
if (newColumns.size() > 0) {
shouldPublish = true;
}
} else {
- RowSignature previousSinkSignature = previousSinkSignatureMap.get(segmentId).lhs;
- Set previousSinkDimensions = new HashSet<>(previousSinkSignature.getColumnNames());
+ RowSignature previousAppendableSegmentSignature = previousAppendableSegmentSignatureMap.get(segmentId).lhs;
+ Set previousAppendableSegmentDimensions = new HashSet<>(
+ previousAppendableSegmentSignature.getColumnNames()
+ );
- Integer previousNumRows = previousSinkSignatureMap.get(segmentId).rhs;
- for (String column : currentSinkSignature.getColumnNames()) {
+ Integer previousNumRows = previousAppendableSegmentSignatureMap.get(segmentId).rhs;
+ for (String column : currentAppendableSegmentSignature.getColumnNames()) {
boolean added = false;
- if (!previousSinkDimensions.contains(column)) {
+ if (!previousAppendableSegmentDimensions.contains(column)) {
newColumns.add(column);
added = true;
- } else if (!Objects.equals(previousSinkSignature.getColumnType(column), currentSinkSignature.getColumnType(column))) {
+ } else if (!Objects.equals(previousAppendableSegmentSignature.getColumnType(column), currentAppendableSegmentSignature.getColumnType(column))) {
updatedColumns.add(column);
added = true;
}
if (added) {
- currentSinkSignature.getColumnType(column).ifPresent(type -> currentColumnMapping.put(column, type));
+ currentAppendableSegmentSignature.getColumnType(column).ifPresent(type -> currentColumnMapping.put(column, type));
}
}
@@ -156,10 +159,10 @@ static Optional computeSchemaChange(
updatedColumns,
currentColumnMapping
);
- sinkSchemas.add(segmentSchema);
+ appendableSegmentSchemas.add(segmentSchema);
}
}
- return Optional.ofNullable(sinkSchemas.isEmpty() ? null : new SegmentSchemas(sinkSchemas));
+ return Optional.ofNullable(appendableSegmentSchemas.isEmpty() ? null : new SegmentSchemas(appendableSegmentSchemas));
}
}
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java
index 734abc6ed542..c73b4f17772c 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java
@@ -73,9 +73,9 @@
import org.apache.druid.segment.loading.DataSegmentPusher;
import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig;
import org.apache.druid.segment.metadata.FingerprintGenerator;
-import org.apache.druid.segment.realtime.FireHydrant;
+import org.apache.druid.segment.realtime.AppendableSegment;
+import org.apache.druid.segment.realtime.PartialSegment;
import org.apache.druid.segment.realtime.SegmentGenerationMetrics;
-import org.apache.druid.segment.realtime.sink.Sink;
import org.apache.druid.server.coordination.DataSegmentAnnouncer;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
@@ -123,10 +123,10 @@ public class AppenderatorImpl implements Appenderator
public static final int ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER = 1000;
public static final int ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER = 700;
public static final int ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER = 600;
- // Rough estimate of memory footprint of empty Sink based on actual heap dumps
- public static final int ROUGH_OVERHEAD_PER_SINK = 5000;
- // Rough estimate of memory footprint of empty FireHydrant based on actual heap dumps
- public static final int ROUGH_OVERHEAD_PER_HYDRANT = 1000;
+ // Rough estimate of memory footprint of empty AppendableSegment based on actual heap dumps
+ public static final int ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT = 5000;
+ // Rough estimate of memory footprint of empty PartialSegment based on actual heap dumps
+ public static final int ROUGH_OVERHEAD_PER_PARTIAL_SEGMENT = 1000;
private static final EmittingLogger log = new EmittingLogger(AppenderatorImpl.class);
private static final int WARN_DELAY = 1000;
@@ -148,9 +148,9 @@ public class AppenderatorImpl implements Appenderator
* called) and from {@link #persistExecutor}. It could also be accessed (but not mutated) potentially in the context
* of any thread from {@link #drop}.
*/
- private final ConcurrentMap sinks = new ConcurrentHashMap<>();
- private final Set droppingSinks = Sets.newConcurrentHashSet();
- private final VersionedIntervalTimeline sinkTimeline;
+ private final ConcurrentMap appendableSegments = new ConcurrentHashMap<>();
+ private final Set droppingAppendableSegments = Sets.newConcurrentHashSet();
+ private final VersionedIntervalTimeline appendableSegmentTimeline;
private final long maxBytesTuningConfig;
private final boolean skipBytesInMemoryOverheadCheck;
@@ -183,15 +183,15 @@ public class AppenderatorImpl implements Appenderator
private final boolean useMaxMemoryEstimates;
/**
- * Use next Map to store metadata (File, SegmentId) for a hydrant for batch appenderator
- * in order to facilitate the mapping of the QueryableIndex associated with a given hydrant
+ * Use next Map to store metadata (File, SegmentId) for a {@link PartialSegment} for batch appenderator
+ * in order to facilitate the mapping of the {@link QueryableIndex} associated with a given PartialSegment
* at merge time. This is necessary since batch appenderator will not map the QueryableIndex
* at persist time in order to minimize its memory footprint. This has to be synchronized since the
* map may be accessed from multiple threads.
* Use {@link IdentityHashMap} to better reflect the fact that the key needs to be interpreted
* with reference semantics.
*/
- private final Map> persistedHydrantMetadata =
+ private final Map> persistedPartialSegmentMetadata =
Collections.synchronizedMap(new IdentityHashMap<>());
private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig;
@@ -199,12 +199,13 @@ public class AppenderatorImpl implements Appenderator
private final FingerprintGenerator fingerprintGenerator;
/**
- * This constructor allows the caller to provide its own SinkQuerySegmentWalker.
- *
- * The sinkTimeline is set to the sink timeline of the provided SinkQuerySegmentWalker.
- * If the SinkQuerySegmentWalker is null, a new sink timeline is initialized.
- *
- * It is used by UnifiedIndexerAppenderatorsManager which allows queries on data associated with multiple
+ * This constructor allows the caller to provide its own {@link AppendableSegmentQuerySegmentWalker}.
+ *
+ * The {@link #appendableSegmentTimeline} is set to the timeline of the provided
+ * {@link AppendableSegmentQuerySegmentWalker}. If the {@link AppendableSegmentQuerySegmentWalker} is null, a new
+ * timeline is initialized.
+ *
+ * It is used by {@link UnifiedIndexerAppenderatorsManager} which allows queries on data associated with multiple
* Appenderators.
*/
AppenderatorImpl(
@@ -215,7 +216,7 @@ public class AppenderatorImpl implements Appenderator
DataSegmentPusher dataSegmentPusher,
ObjectMapper objectMapper,
DataSegmentAnnouncer segmentAnnouncer,
- @Nullable SinkQuerySegmentWalker sinkQuerySegmentWalker,
+ @Nullable AppendableSegmentQuerySegmentWalker appendableSegmentQuerySegmentWalker,
IndexIO indexIO,
IndexMerger indexMerger,
Cache cache,
@@ -236,18 +237,18 @@ public class AppenderatorImpl implements Appenderator
this.indexIO = Preconditions.checkNotNull(indexIO, "indexIO");
this.indexMerger = Preconditions.checkNotNull(indexMerger, "indexMerger");
this.cache = cache;
- this.texasRanger = sinkQuerySegmentWalker;
+ this.texasRanger = appendableSegmentQuerySegmentWalker;
this.rowIngestionMeters = Preconditions.checkNotNull(rowIngestionMeters, "rowIngestionMeters");
this.parseExceptionHandler = Preconditions.checkNotNull(parseExceptionHandler, "parseExceptionHandler");
this.isOpenSegments = isOpenSegments;
this.useMaxMemoryEstimates = useMaxMemoryEstimates;
- if (sinkQuerySegmentWalker == null) {
- this.sinkTimeline = new VersionedIntervalTimeline<>(
+ if (appendableSegmentQuerySegmentWalker == null) {
+ this.appendableSegmentTimeline = new VersionedIntervalTimeline<>(
String.CASE_INSENSITIVE_ORDER
);
} else {
- this.sinkTimeline = sinkQuerySegmentWalker.getSinkTimeline();
+ this.appendableSegmentTimeline = appendableSegmentQuerySegmentWalker.getAppendableSegmentTimeline();
}
maxBytesTuningConfig = tuningConfig.getMaxBytesInMemoryOrDefault();
@@ -278,7 +279,7 @@ public String getDataSource()
public Object startJob()
{
lockBasePersistDirectory();
- final Object retVal = bootstrapSinksFromDisk();
+ final Object retVal = bootstrapAppendableSegmentsFromDisk();
initializeExecutors();
resetNextFlush();
return retVal;
@@ -309,29 +310,29 @@ public AppenderatorAddResult add(
);
}
- final Sink sink = getOrCreateSink(identifier);
+ final AppendableSegment appendableSegment = getOrCreateAppendableSegment(identifier);
metrics.reportMessageMaxTimestamp(row.getTimestampFromEpoch());
- final int sinkRowsInMemoryBeforeAdd = sink.getNumRowsInMemory();
- final int sinkRowsInMemoryAfterAdd;
- final long bytesInMemoryBeforeAdd = sink.getBytesInMemory();
+ final int appendableSegmentRowsInMemoryBeforeAdd = appendableSegment.getNumRowsInMemory();
+ final int appendableSegmentRowsInMemoryAfterAdd;
+ final long bytesInMemoryBeforeAdd = appendableSegment.getBytesInMemory();
final long bytesInMemoryAfterAdd;
final IncrementalIndexAddResult addResult;
try {
- addResult = sink.add(row, !allowIncrementalPersists);
- sinkRowsInMemoryAfterAdd = addResult.getRowCount();
+ addResult = appendableSegment.add(row, !allowIncrementalPersists);
+ appendableSegmentRowsInMemoryAfterAdd = addResult.getRowCount();
bytesInMemoryAfterAdd = addResult.getBytesInMemory();
}
catch (IndexSizeExceededException e) {
// Uh oh, we can't do anything about this! We can't persist (commit metadata would be out of sync) and we
// can't add the row (it just failed). This should never actually happen, though, because we check
- // sink.canAddRow after returning from add.
- log.error(e, "Sink for segment[%s] was unexpectedly full!", identifier);
+ // appendableSegment.canAddRow after returning from add.
+ log.error(e, "Appendable segment[%s] was unexpectedly full!", identifier);
throw e;
}
- if (sinkRowsInMemoryAfterAdd < 0) {
- throw new SegmentNotWritableException("Attempt to add row to swapped-out sink for segment[%s].", identifier);
+ if (appendableSegmentRowsInMemoryAfterAdd < 0) {
+ throw new SegmentNotWritableException("Attempt to add row to swapped-out appendable segment[%s].", identifier);
}
if (addResult.isRowAdded()) {
@@ -340,7 +341,7 @@ public AppenderatorAddResult add(
parseExceptionHandler.handle(addResult.getParseException());
}
- final int numAddedRows = sinkRowsInMemoryAfterAdd - sinkRowsInMemoryBeforeAdd;
+ final int numAddedRows = appendableSegmentRowsInMemoryAfterAdd - appendableSegmentRowsInMemoryBeforeAdd;
rowsCurrentlyInMemory.addAndGet(numAddedRows);
bytesCurrentlyInMemory.addAndGet(bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd);
totalRows.addAndGet(numAddedRows);
@@ -349,9 +350,9 @@ public AppenderatorAddResult add(
boolean persist = false;
List persistReasons = new ArrayList<>();
- if (!sink.canAppendRow()) {
+ if (!appendableSegment.canAppendRow()) {
persist = true;
- persistReasons.add("No more rows can be appended to sink");
+ persistReasons.add("No more rows can be appended to appendable segment");
}
if (System.currentTimeMillis() > nextFlush) {
persist = true;
@@ -383,15 +384,18 @@ public AppenderatorAddResult add(
log.info("Flushing in-memory data to disk because %s.", String.join(",", persistReasons));
long bytesToBePersisted = 0L;
- for (Map.Entry entry : sinks.entrySet()) {
- final Sink sinkEntry = entry.getValue();
- if (sinkEntry != null) {
- bytesToBePersisted += sinkEntry.getBytesInMemory();
- if (sinkEntry.swappable()) {
- // After swapping the sink, we use memory mapped segment instead (but only for real time appenderators!).
- // However, the memory mapped segment still consumes memory.
- // These memory mapped segments are held in memory throughout the ingestion phase and permanently add to the bytesCurrentlyInMemory
- int memoryStillInUse = calculateMMappedHydrantMemoryInUsed(sink.getCurrHydrant());
+ for (Map.Entry entry : appendableSegments.entrySet()) {
+ final AppendableSegment appendableSegmentEntry = entry.getValue();
+ if (appendableSegmentEntry != null) {
+ bytesToBePersisted += appendableSegmentEntry.getBytesInMemory();
+ if (appendableSegmentEntry.swappable()) {
+ // After swapping the appendable segment, we use memory mapped segment instead (but only for real time
+ // appenderators!). However, the memory mapped segment still consumes memory. These memory mapped
+ // segments are held in memory throughout the ingestion phase and permanently add to the
+ // bytesCurrentlyInMemory
+ int memoryStillInUse = calculateMMappedPartialSegmentMemoryInUsed(
+ appendableSegment.getCurrentPartialSegment()
+ );
bytesCurrentlyInMemory.addAndGet(memoryStillInUse);
}
}
@@ -403,10 +407,10 @@ public AppenderatorAddResult add(
// This means that we ran out of all available memory to ingest (due to overheads created as part of ingestion)
final String alertMessage = StringUtils.format(
"Task has exceeded safe estimated heap usage limits, failing "
- + "(numSinks: [%d] numHydrantsAcrossAllSinks: [%d] totalRows: [%d])"
+ + "(numAppendableSegments: [%d] numParialSegmentsAcrossAllAppendableSegments: [%d] totalRows: [%d])"
+ "(bytesCurrentlyInMemory: [%d] - bytesToBePersisted: [%d] > maxBytesTuningConfig: [%d])",
- sinks.size(),
- sinks.values().stream().mapToInt(Iterables::size).sum(),
+ appendableSegments.size(),
+ appendableSegments.values().stream().mapToInt(Iterables::size).sum(),
getTotalRowCount(),
bytesCurrentlyInMemory.get(),
bytesToBePersisted,
@@ -450,24 +454,24 @@ public void onFailure(Throwable t)
isPersistRequired = true;
}
}
- return new AppenderatorAddResult(identifier, sink.getNumRows(), isPersistRequired);
+ return new AppenderatorAddResult(identifier, appendableSegment.getNumRows(), isPersistRequired);
}
@Override
public List getSegments()
{
- return ImmutableList.copyOf(sinks.keySet());
+ return ImmutableList.copyOf(appendableSegments.keySet());
}
@Override
public int getRowCount(final SegmentIdWithShardSpec identifier)
{
- final Sink sink = sinks.get(identifier);
+ final AppendableSegment appendableSegment = appendableSegments.get(identifier);
- if (sink == null) {
- throw new ISE("No such sink: %s", identifier);
+ if (appendableSegment == null) {
+ throw new ISE("No such apendable segment: %s", identifier);
} else {
- return sink.getNumRows();
+ return appendableSegment.getNumRows();
}
}
@@ -492,21 +496,21 @@ long getBytesCurrentlyInMemory()
@VisibleForTesting
long getBytesInMemory(SegmentIdWithShardSpec identifier)
{
- final Sink sink = sinks.get(identifier);
+ final AppendableSegment appendableSegment = appendableSegments.get(identifier);
- if (sink == null) {
- throw new ISE("No such sink: %s", identifier);
+ if (appendableSegment == null) {
+ throw new ISE("No such appendable segment: %s", identifier);
} else {
- return sink.getBytesInMemory();
+ return appendableSegment.getBytesInMemory();
}
}
- private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier)
+ private AppendableSegment getOrCreateAppendableSegment(final SegmentIdWithShardSpec identifier)
{
- Sink retVal = sinks.get(identifier);
+ AppendableSegment retVal = appendableSegments.get(identifier);
if (retVal == null) {
- retVal = new Sink(
+ retVal = new AppendableSegment(
identifier.getInterval(),
schema,
identifier.getShardSpec(),
@@ -516,7 +520,7 @@ private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier)
maxBytesTuningConfig,
useMaxMemoryEstimates
);
- bytesCurrentlyInMemory.addAndGet(calculateSinkMemoryInUsed());
+ bytesCurrentlyInMemory.addAndGet(calculateAppendableSegmentMemoryInUsed());
try {
segmentAnnouncer.announceSegment(retVal.getSegment());
@@ -527,9 +531,13 @@ private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier)
.emit();
}
- sinks.put(identifier, retVal);
- metrics.setSinkCount(sinks.size());
- sinkTimeline.add(retVal.getInterval(), retVal.getVersion(), identifier.getShardSpec().createChunk(retVal));
+ appendableSegments.put(identifier, retVal);
+ metrics.setAppendableSegmentCount(appendableSegments.size());
+ appendableSegmentTimeline.add(
+ retVal.getInterval(),
+ retVal.getVersion(),
+ identifier.getShardSpec().createChunk(retVal)
+ );
}
return retVal;
@@ -582,12 +590,12 @@ public void clear() throws InterruptedException
// Drop everything.
final List> futures = new ArrayList<>();
- for (Map.Entry entry : sinks.entrySet()) {
+ for (Map.Entry entry : appendableSegments.entrySet()) {
futures.add(abandonSegment(entry.getKey(), entry.getValue(), true));
}
- // Re-initialize hydrant map:
- persistedHydrantMetadata.clear();
+ // Re-initialize partial segment map:
+ persistedPartialSegmentMetadata.clear();
// Await dropping.
Futures.allAsList(futures).get();
@@ -601,9 +609,9 @@ public void clear() throws InterruptedException
@Override
public ListenableFuture> drop(final SegmentIdWithShardSpec identifier)
{
- final Sink sink = sinks.get(identifier);
- if (sink != null) {
- return abandonSegment(identifier, sink, true);
+ final AppendableSegment appendableSegment = appendableSegments.get(identifier);
+ if (appendableSegment != null) {
+ return abandonSegment(identifier, appendableSegment, true);
} else {
return Futures.immediateFuture(null);
}
@@ -613,40 +621,40 @@ public ListenableFuture> drop(final SegmentIdWithShardSpec identifier)
public ListenableFuture persistAll(@Nullable final Committer committer)
{
throwPersistErrorIfExists();
- final Map currentHydrants = new HashMap<>();
- final List> indexesToPersist = new ArrayList<>();
+ final Map currentPartialSegments = new HashMap<>();
+ final List> indexesToPersist = new ArrayList<>();
int numPersistedRows = 0;
long bytesPersisted = 0L;
- MutableLong totalHydrantsCount = new MutableLong();
- MutableLong totalHydrantsPersisted = new MutableLong();
- final long totalSinks = sinks.size();
- for (Map.Entry entry : sinks.entrySet()) {
+ MutableLong totalPartialSegmentsCount = new MutableLong();
+ MutableLong totalPartialSegmentsPersisted = new MutableLong();
+ final long totalAppendableSegments = appendableSegments.size();
+ for (Map.Entry entry : appendableSegments.entrySet()) {
final SegmentIdWithShardSpec identifier = entry.getKey();
- final Sink sink = entry.getValue();
- if (sink == null) {
- throw new ISE("No sink for identifier: %s", identifier);
+ final AppendableSegment appendableSegment = entry.getValue();
+ if (appendableSegment == null) {
+ throw new ISE("No appendable segment for identifier: %s", identifier);
}
- final List hydrants = Lists.newArrayList(sink);
- totalHydrantsCount.add(hydrants.size());
- currentHydrants.put(identifier.toString(), hydrants.size());
- numPersistedRows += sink.getNumRowsInMemory();
- bytesPersisted += sink.getBytesInMemory();
-
- final int limit = sink.isWritable() ? hydrants.size() - 1 : hydrants.size();
-
- // gather hydrants that have not been persisted:
- for (FireHydrant hydrant : hydrants.subList(0, limit)) {
- if (!hydrant.hasSwapped()) {
- log.debug("Hydrant[%s] hasn't persisted yet, persisting. Segment[%s]", hydrant, identifier);
- indexesToPersist.add(Pair.of(hydrant, identifier));
- totalHydrantsPersisted.add(1);
+ final List partialSegments = Lists.newArrayList(appendableSegment);
+ totalPartialSegmentsCount.add(partialSegments.size());
+ currentPartialSegments.put(identifier.toString(), partialSegments.size());
+ numPersistedRows += appendableSegment.getNumRowsInMemory();
+ bytesPersisted += appendableSegment.getBytesInMemory();
+
+ final int limit = appendableSegment.isWritable() ? partialSegments.size() - 1 : partialSegments.size();
+
+ // gather partialSegments that have not been persisted:
+ for (PartialSegment partialSegment : partialSegments.subList(0, limit)) {
+ if (!partialSegment.hasSwapped()) {
+ log.debug("Partial segment[%s] hasn't persisted yet, persisting. Segment[%s]", partialSegment, identifier);
+ indexesToPersist.add(Pair.of(partialSegment, identifier));
+ totalPartialSegmentsPersisted.add(1);
}
}
- if (sink.swappable()) {
+ if (appendableSegment.swappable()) {
// It is swappable. Get the old one to persist it and create a new one:
- indexesToPersist.add(Pair.of(sink.swap(), identifier));
- totalHydrantsPersisted.add(1);
+ indexesToPersist.add(Pair.of(appendableSegment.swap(), identifier));
+ totalPartialSegmentsPersisted.add(1);
}
}
log.debug("Submitting persist runnable for dataSource[%s]", schema.getDataSource());
@@ -662,23 +670,23 @@ public ListenableFuture persistAll(@Nullable final Committer committer)
public Object call() throws IOException
{
try {
- for (Pair pair : indexesToPersist) {
- metrics.incrementRowOutputCount(persistHydrant(pair.lhs, pair.rhs));
+ for (Pair pair : indexesToPersist) {
+ metrics.incrementRowOutputCount(persistPartialSegment(pair.lhs, pair.rhs));
}
if (committer != null) {
log.debug(
- "Committing metadata[%s] for sinks[%s].",
+ "Committing metadata[%s] for appendable segments[%s].",
commitMetadata,
Joiner.on(", ").join(
- currentHydrants.entrySet()
- .stream()
- .map(entry -> StringUtils.format(
- "%s:%d",
- entry.getKey(),
- entry.getValue()
- ))
- .collect(Collectors.toList())
+ currentPartialSegments.entrySet()
+ .stream()
+ .map(entry -> StringUtils.format(
+ "%s:%d",
+ entry.getKey(),
+ entry.getValue()
+ ))
+ .collect(Collectors.toList())
)
);
@@ -686,14 +694,14 @@ public Object call() throws IOException
try {
commitLock.lock();
- final Map commitHydrants = new HashMap<>();
+ final Map commitPartialSegments = new HashMap<>();
final Committed oldCommit = readCommit();
if (oldCommit != null) {
- // merge current hydrants with existing hydrants
- commitHydrants.putAll(oldCommit.getHydrants());
+ // merge current PartialSegment with existing PartialSegment
+ commitPartialSegments.putAll(oldCommit.getPartialSegments());
}
- commitHydrants.putAll(currentHydrants);
- writeCommit(new Committed(commitHydrants, commitMetadata));
+ commitPartialSegments.putAll(currentPartialSegments);
+ writeCommit(new Committed(commitPartialSegments, commitMetadata));
}
finally {
commitLock.unlock();
@@ -709,12 +717,12 @@ public Object call() throws IOException
.collect(Collectors.joining(", "))
);
log.info(
- "Persisted stats: processed rows: [%d], persisted rows[%d], sinks: [%d], total fireHydrants (across sinks): [%d], persisted fireHydrants (across sinks): [%d]",
+ "Persisted stats: processed rows: [%d], persisted rows[%d], appendable segments: [%d], total partial segments (across appendable segments): [%d], persisted partial segments (across appendable segments): [%d]",
rowIngestionMeters.getProcessed(),
totalPersistedRows.get(),
- totalSinks,
- totalHydrantsCount.longValue(),
- totalHydrantsPersisted.longValue()
+ totalAppendableSegments,
+ totalPartialSegmentsCount.longValue(),
+ totalPartialSegmentsPersisted.longValue()
);
// return null if committer is null
@@ -757,19 +765,19 @@ public ListenableFuture push(
final boolean useUniquePath
)
{
- final Map theSinks = new HashMap<>();
- AtomicLong pushedHydrantsCount = new AtomicLong();
+ final Map theAppendableSegments = new HashMap<>();
+ AtomicLong pushedPartialSegmentsCount = new AtomicLong();
for (final SegmentIdWithShardSpec identifier : identifiers) {
- final Sink sink = sinks.get(identifier);
- if (sink == null) {
- throw new ISE("No sink for identifier: %s", identifier);
+ final AppendableSegment appendableSegment = appendableSegments.get(identifier);
+ if (appendableSegment == null) {
+ throw new ISE("No appendable segment for identifier: %s", identifier);
}
- theSinks.put(identifier, sink);
- if (sink.finishWriting()) {
- totalRows.addAndGet(-sink.getNumRows());
+ theAppendableSegments.put(identifier, appendableSegment);
+ if (appendableSegment.finishWriting()) {
+ totalRows.addAndGet(-appendableSegment.getNumRows());
}
- // count hydrants for stats:
- pushedHydrantsCount.addAndGet(Iterables.size(sink));
+ // count partial segments for stats:
+ pushedPartialSegmentsCount.addAndGet(Iterables.size(appendableSegment));
}
return Futures.transform(
@@ -780,18 +788,24 @@ public ListenableFuture push(
final List dataSegments = new ArrayList<>();
final SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION);
- log.info("Preparing to push (stats): processed rows: [%d], sinks: [%d], fireHydrants (across sinks): [%d]",
- rowIngestionMeters.getProcessed(), theSinks.size(), pushedHydrantsCount.get()
+ log.info(
+ "Preparing to push (stats): processed rows: [%d], appendable segments: [%d], partial segments (across appendable segments): [%d]",
+ rowIngestionMeters.getProcessed(),
+ theAppendableSegments.size(),
+ pushedPartialSegmentsCount.get()
);
log.debug(
"Building and pushing segments: %s",
- theSinks.keySet().stream().map(SegmentIdWithShardSpec::toString).collect(Collectors.joining(", "))
+ theAppendableSegments.keySet()
+ .stream()
+ .map(SegmentIdWithShardSpec::toString)
+ .collect(Collectors.joining(", "))
);
- for (Map.Entry entry : theSinks.entrySet()) {
- if (droppingSinks.contains(entry.getKey())) {
- log.warn("Skipping push of currently-dropping sink[%s]", entry.getKey());
+ for (Map.Entry entry : theAppendableSegments.entrySet()) {
+ if (droppingAppendableSegments.contains(entry.getKey())) {
+ log.warn("Skipping push of currently-dropping appendable segment[%s]", entry.getKey());
continue;
}
@@ -838,7 +852,8 @@ public ListenableFuture push(
private ListenableFuture> pushBarrier()
{
return intermediateTempExecutor.submit(
- (Runnable) () -> pushExecutor.submit(() -> {})
+ (Runnable) () -> pushExecutor.submit(() -> {
+ })
);
}
@@ -846,23 +861,22 @@ private ListenableFuture> pushBarrier()
* Merge segment, push to deep storage. Should only be used on segments that have been fully persisted. Must only
* be run in the single-threaded pushExecutor.
*
- * @param identifier sink identifier
- * @param sink sink to push
- * @param useUniquePath true if the segment should be written to a path with a unique identifier
- *
- * @return segment descriptor, or null if the sink is no longer valid
+ * @param identifier appendable segment identifier
+ * @param appendableSegment appendable segment to push
+ * @param useUniquePath true if the segment should be written to a path with a unique identifier
+ * @return segment descriptor, or null if the appendable segment is no longer valid
*/
@Nullable
private DataSegmentWithMetadata mergeAndPush(
final SegmentIdWithShardSpec identifier,
- final Sink sink,
+ final AppendableSegment appendableSegment,
final boolean useUniquePath
)
{
- // Bail out if this sink is null or otherwise not what we expect.
+ // Bail out if this appendable segment is null or otherwise not what we expect.
//noinspection ObjectEquality
- if (sinks.get(identifier) != sink) {
- log.warn("Sink for segment[%s] no longer valid, bailing out of mergeAndPush.", identifier);
+ if (appendableSegments.get(identifier) != appendableSegment) {
+ log.warn("Appendable segment[%s] no longer valid, bailing out of mergeAndPush.", identifier);
return null;
}
@@ -872,14 +886,20 @@ private DataSegmentWithMetadata mergeAndPush(
final File descriptorFile = computeDescriptorFile(identifier);
// Sanity checks
- for (FireHydrant hydrant : sink) {
- if (sink.isWritable()) {
- throw new ISE("Expected sink to be no longer writable before mergeAndPush for segment[%s].", identifier);
+ for (PartialSegment partialSegment : appendableSegment) {
+ if (appendableSegment.isWritable()) {
+ throw new ISE(
+ "Expected appendable segment to be no longer writable before mergeAndPush for segment[%s].",
+ identifier
+ );
}
- synchronized (hydrant) {
- if (!hydrant.hasSwapped()) {
- throw new ISE("Expected sink to be fully persisted before mergeAndPush for segment[%s].", identifier);
+ synchronized (partialSegment) {
+ if (!partialSegment.hasSwapped()) {
+ throw new ISE(
+ "Expected appendable segment to be fully persisted before mergeAndPush for segment[%s].",
+ identifier
+ );
}
}
}
@@ -919,15 +939,15 @@ private DataSegmentWithMetadata mergeAndPush(
List indexes = new ArrayList<>();
Closer closer = Closer.create();
try {
- for (FireHydrant fireHydrant : sink) {
+ for (PartialSegment partialSegment : appendableSegment) {
// if batch, swap/persist did not memory map the incremental index, we need it mapped now:
if (!isOpenSegments()) {
// sanity
- Pair persistedMetadata = persistedHydrantMetadata.get(fireHydrant);
+ Pair persistedMetadata = persistedPartialSegmentMetadata.get(partialSegment);
if (persistedMetadata == null) {
- throw new ISE("Persisted metadata for batch hydrant [%s] is null!", fireHydrant);
+ throw new ISE("Persisted metadata for batch partial segment [%s] is null!", partialSegment);
}
File persistedFile = persistedMetadata.lhs;
@@ -935,22 +955,22 @@ private DataSegmentWithMetadata mergeAndPush(
// sanity:
if (persistedFile == null) {
- throw new ISE("Persisted file for batch hydrant [%s] is null!", fireHydrant);
+ throw new ISE("Persisted file for batch partial segment [%s] is null!", partialSegment);
} else if (persistedSegmentId == null) {
throw new ISE(
- "Persisted segmentId for batch hydrant in file [%s] is null!",
+ "Persisted segmentId for batch partial segment in file [%s] is null!",
persistedFile.getPath()
);
}
- fireHydrant.swapSegment(new QueryableIndexSegment(
+ partialSegment.swapSegment(new QueryableIndexSegment(
indexIO.loadIndex(persistedFile),
persistedSegmentId
));
}
- Pair segmentAndCloseable = fireHydrant.getAndIncrementSegment();
+ Pair segmentAndCloseable = partialSegment.getAndIncrementSegment();
final QueryableIndex queryableIndex = segmentAndCloseable.lhs.asQueryableIndex();
- log.debug("Segment[%s] adding hydrant[%s]", identifier, fireHydrant);
+ log.debug("Segment[%s] adding partial segment[%s]", identifier, partialSegment);
indexes.add(queryableIndex);
closer.register(segmentAndCloseable.rhs);
}
@@ -979,7 +999,7 @@ private DataSegmentWithMetadata mergeAndPush(
closer.close();
}
- final DataSegment segmentToPush = sink.getSegment().withDimensions(
+ final DataSegment segmentToPush = appendableSegment.getSegment().withDimensions(
IndexMerger.getMergedDimensionsFromQueryableIndexes(indexes, schema.getDimensionsSpec())
);
@@ -991,11 +1011,11 @@ private DataSegmentWithMetadata mergeAndPush(
final DataSegment segment = dataSegmentPusher.push(mergedFile, segmentToPush, useUniquePath);
if (!isOpenSegments()) {
- // Drop the queryable indexes behind the hydrants... they are not needed anymore and their
+ // Drop the queryable indexes behind the partial segments... they are not needed anymore and their
// mapped file references
// can generate OOMs during merge if enough of them are held back...
- for (FireHydrant fireHydrant : sink) {
- fireHydrant.swapSegment(null);
+ for (PartialSegment partialSegment : appendableSegment) {
+ partialSegment.swapSegment(null);
}
}
@@ -1041,7 +1061,7 @@ public void close()
log.debug("Shutting down...");
final List> futures = new ArrayList<>();
- for (Map.Entry entry : sinks.entrySet()) {
+ for (Map.Entry entry : appendableSegments.entrySet()) {
futures.add(abandonSegment(entry.getKey(), entry.getValue(), false));
}
@@ -1100,7 +1120,7 @@ public void closeNow()
}
log.debug("Shutting down immediately...");
- for (Map.Entry entry : sinks.entrySet()) {
+ for (Map.Entry entry : appendableSegments.entrySet()) {
try {
segmentAnnouncer.unannounceSegment(entry.getValue().getSegment());
}
@@ -1222,13 +1242,14 @@ private void resetNextFlush()
}
/**
- * Populate "sinks" and "sinkTimeline" with committed segments, and announce them with the segmentAnnouncer.
+ * Populate {@link #appendableSegments} and {@link #appendableSegmentTimeline} with committed segments, and
+ * announce them with the segmentAnnouncer.
*
* @return persisted commit metadata
*/
- private Object bootstrapSinksFromDisk()
+ private Object bootstrapAppendableSegmentsFromDisk()
{
- Preconditions.checkState(sinks.isEmpty(), "Already bootstrapped?!");
+ Preconditions.checkState(appendableSegments.isEmpty(), "Already bootstrapped?!");
final File baseDir = tuningConfig.getBasePersistDirectory();
if (!baseDir.exists()) {
@@ -1266,71 +1287,79 @@ private Object bootstrapSinksFromDisk()
} else {
log.info(
"Loading partially-persisted segments[%s] from[%s] with commit metadata: %s",
- String.join(", ", committed.getHydrants().keySet()),
+ String.join(", ", committed.getPartialSegments().keySet()),
baseDir,
committed.getMetadata()
);
}
- for (File sinkDir : files) {
- final File identifierFile = new File(sinkDir, IDENTIFIER_FILE_NAME);
+ for (File appendableSegmentDir : files) {
+ final File identifierFile = new File(appendableSegmentDir, IDENTIFIER_FILE_NAME);
if (!identifierFile.isFile()) {
- // No identifier in this sinkDir; it must not actually be a sink directory. Skip it.
+ // No identifier in this appendableSegmentDir; it must not actually be a appendable segment directory. Skip it.
continue;
}
try {
final SegmentIdWithShardSpec identifier = objectMapper.readValue(
- new File(sinkDir, "identifier.json"),
+ new File(appendableSegmentDir, "identifier.json"),
SegmentIdWithShardSpec.class
);
- final int committedHydrants = committed.getCommittedHydrants(identifier.toString());
+ final int committedPartialSegments = committed.getCommittedPartialSegments(identifier.toString());
- if (committedHydrants <= 0) {
- log.info("Removing uncommitted segment at [%s].", sinkDir);
- FileUtils.deleteDirectory(sinkDir);
+ if (committedPartialSegments <= 0) {
+ log.info("Removing uncommitted segment at [%s].", appendableSegmentDir);
+ FileUtils.deleteDirectory(appendableSegmentDir);
continue;
}
// To avoid reading and listing of "merged" dir and other special files
- final File[] sinkFiles = sinkDir.listFiles(
+ final File[] appendableSegmentFiles = appendableSegmentDir.listFiles(
(dir, fileName) -> !(Ints.tryParse(fileName) == null)
);
Arrays.sort(
- sinkFiles,
+ appendableSegmentFiles,
(o1, o2) -> Ints.compare(Integer.parseInt(o1.getName()), Integer.parseInt(o2.getName()))
);
- List hydrants = new ArrayList<>();
- for (File hydrantDir : sinkFiles) {
- final int hydrantNumber = Integer.parseInt(hydrantDir.getName());
+ List partialSegments = new ArrayList<>();
+ for (File partialSegmentDir : appendableSegmentFiles) {
+ final int partialSegmentNumber = Integer.parseInt(partialSegmentDir.getName());
- if (hydrantNumber >= committedHydrants) {
- log.info("Removing uncommitted partial segment at [%s]", hydrantDir);
- FileUtils.deleteDirectory(hydrantDir);
+ if (partialSegmentNumber >= committedPartialSegments) {
+ log.info("Removing uncommitted partial segment at [%s]", partialSegmentDir);
+ FileUtils.deleteDirectory(partialSegmentDir);
} else {
- log.debug("Loading previously persisted partial segment at [%s]", hydrantDir);
- if (hydrantNumber != hydrants.size()) {
- throw new ISE("Missing hydrant [%,d] in sinkDir [%s].", hydrants.size(), sinkDir);
+ log.debug("Loading previously persisted partial segment at [%s]", partialSegmentDir);
+ if (partialSegmentNumber != partialSegments.size()) {
+ throw new ISE(
+ "Missing partial segment [%,d] in appendableSegmentDir [%s].",
+ partialSegments.size(),
+ appendableSegmentDir
+ );
}
- hydrants.add(
- new FireHydrant(
- new QueryableIndexSegment(indexIO.loadIndex(hydrantDir), identifier.asSegmentId()),
- hydrantNumber
+ partialSegments.add(
+ new PartialSegment(
+ new QueryableIndexSegment(indexIO.loadIndex(partialSegmentDir), identifier.asSegmentId()),
+ partialSegmentNumber
)
);
}
}
- // Make sure we loaded enough hydrants.
- if (committedHydrants != hydrants.size()) {
- throw new ISE("Missing hydrant [%,d] in sinkDir [%s].", hydrants.size(), sinkDir);
+ // Make sure we loaded enough partialSegments.
+ if (committedPartialSegments != partialSegments.size()) {
+ throw new ISE(
+ "Missing partial segment [%,d] in appendableSegmentDir [%s].",
+ partialSegments.size(),
+ appendableSegmentDir
+ );
}
- Sink currSink = new Sink(
+ AppendableSegment currAppendableSegment = new AppendableSegment(
identifier.getInterval(),
schema,
identifier.getShardSpec(),
@@ -1339,32 +1368,35 @@ private Object bootstrapSinksFromDisk()
tuningConfig.getMaxRowsInMemory(),
maxBytesTuningConfig,
useMaxMemoryEstimates,
- hydrants
+ partialSegments
);
- rowsSoFar += currSink.getNumRows();
- sinks.put(identifier, currSink);
- sinkTimeline.add(
- currSink.getInterval(),
- currSink.getVersion(),
- identifier.getShardSpec().createChunk(currSink)
+ rowsSoFar += currAppendableSegment.getNumRows();
+ appendableSegments.put(identifier, currAppendableSegment);
+ appendableSegmentTimeline.add(
+ currAppendableSegment.getInterval(),
+ currAppendableSegment.getVersion(),
+ identifier.getShardSpec().createChunk(currAppendableSegment)
);
- segmentAnnouncer.announceSegment(currSink.getSegment());
+ segmentAnnouncer.announceSegment(currAppendableSegment.getSegment());
}
catch (IOException e) {
- log.makeAlert(e, "Problem loading sink[%s] from disk.", schema.getDataSource())
- .addData("sinkDir", sinkDir)
+ log.makeAlert(e, "Problem loading appendable segment[%s] from disk.", schema.getDataSource())
+ .addData("appendableSegmentDir", appendableSegmentDir)
.emit();
}
}
- // Make sure we loaded all committed sinks.
- final Set loadedSinks = Sets.newHashSet(
- Iterables.transform(sinks.keySet(), SegmentIdWithShardSpec::toString)
+ // Make sure we loaded all committed appendable segments.
+ final Set loadedAppendableSegments = Sets.newHashSet(
+ Iterables.transform(appendableSegments.keySet(), SegmentIdWithShardSpec::toString)
+ );
+ final Set missingAppendableSegments = Sets.difference(
+ committed.getPartialSegments().keySet(),
+ loadedAppendableSegments
);
- final Set missingSinks = Sets.difference(committed.getHydrants().keySet(), loadedSinks);
- if (!missingSinks.isEmpty()) {
- throw new ISE("Missing committed sinks [%s]", Joiner.on(", ").join(missingSinks));
+ if (!missingAppendableSegments.isEmpty()) {
+ throw new ISE("Missing committed appendable segments [%s]", Joiner.on(", ").join(missingAppendableSegments));
}
totalRows.set(rowsSoFar);
@@ -1373,28 +1405,29 @@ private Object bootstrapSinksFromDisk()
private ListenableFuture> abandonSegment(
final SegmentIdWithShardSpec identifier,
- final Sink sink,
+ final AppendableSegment appendableSegment,
final boolean removeOnDiskData
)
{
- // Ensure no future writes will be made to this sink.
- if (sink.finishWriting()) {
- // Decrement this sink's rows from the counters. we only count active sinks so that we don't double decrement,
- // i.e. those that haven't been persisted for *InMemory counters, or pushed to deep storage for the total counter.
- rowsCurrentlyInMemory.addAndGet(-sink.getNumRowsInMemory());
- bytesCurrentlyInMemory.addAndGet(-sink.getBytesInMemory());
- bytesCurrentlyInMemory.addAndGet(-calculateSinkMemoryInUsed());
- for (FireHydrant hydrant : sink) {
- // Decrement memory used by all Memory Mapped Hydrant
- if (!hydrant.equals(sink.getCurrHydrant())) {
- bytesCurrentlyInMemory.addAndGet(-calculateMMappedHydrantMemoryInUsed(hydrant));
+ // Ensure no future writes will be made to this appendable segment.
+ if (appendableSegment.finishWriting()) {
+ // Decrement this appendable segment's rows from the counters. we only count active appendable segment so that we
+ // don't double decrement, i.e. those that haven't been persisted for *InMemory counters, or pushed to deep
+ // storage for the total counter.
+ rowsCurrentlyInMemory.addAndGet(-appendableSegment.getNumRowsInMemory());
+ bytesCurrentlyInMemory.addAndGet(-appendableSegment.getBytesInMemory());
+ bytesCurrentlyInMemory.addAndGet(-calculateAppendableSegmentMemoryInUsed());
+ for (PartialSegment partialSegment : appendableSegment) {
+ // Decrement memory used by all Memory Mapped PartialSegment
+ if (!partialSegment.equals(appendableSegment.getCurrentPartialSegment())) {
+ bytesCurrentlyInMemory.addAndGet(-calculateMMappedPartialSegmentMemoryInUsed(partialSegment));
}
}
- totalRows.addAndGet(-sink.getNumRows());
+ totalRows.addAndGet(-appendableSegment.getNumRows());
}
// Mark this identifier as dropping, so no future push tasks will pick it up.
- droppingSinks.add(identifier);
+ droppingAppendableSegments.add(identifier);
// Wait for any outstanding pushes to finish, then abandon the segment inside the persist thread.
return Futures.transform(
@@ -1405,12 +1438,12 @@ private ListenableFuture> abandonSegment(
@Override
public Void apply(@Nullable Object input)
{
- if (!sinks.remove(identifier, sink)) {
- log.error("Sink for segment[%s] no longer valid, not abandoning.", identifier);
+ if (!appendableSegments.remove(identifier, appendableSegment)) {
+ log.error("Appendable segment[%s] no longer valid, not abandoning.", identifier);
return null;
}
- metrics.setSinkCount(sinks.size());
+ metrics.setAppendableSegmentCount(appendableSegments.size());
if (removeOnDiskData) {
// Remove this segment from the committed list. This must be done from the persist thread.
@@ -1435,7 +1468,7 @@ public Void apply(@Nullable Object input)
// Unannounce the segment.
try {
- segmentAnnouncer.unannounceSegment(sink.getSegment());
+ segmentAnnouncer.unannounceSegment(appendableSegment.getSegment());
}
catch (Exception e) {
log.makeAlert(e, "Failed to unannounce segment[%s]", schema.getDataSource())
@@ -1443,19 +1476,19 @@ public Void apply(@Nullable Object input)
.emit();
}
- droppingSinks.remove(identifier);
- sinkTimeline.remove(
- sink.getInterval(),
- sink.getVersion(),
- identifier.getShardSpec().createChunk(sink)
+ droppingAppendableSegments.remove(identifier);
+ appendableSegmentTimeline.remove(
+ appendableSegment.getInterval(),
+ appendableSegment.getVersion(),
+ identifier.getShardSpec().createChunk(appendableSegment)
);
- for (FireHydrant hydrant : sink) {
+ for (PartialSegment partialSegment : appendableSegment) {
if (cache != null) {
- cache.close(SinkQuerySegmentWalker.makeHydrantCacheIdentifier(hydrant));
+ cache.close(AppendableSegmentQuerySegmentWalker.makePartialSegmentCacheIdentifier(partialSegment));
}
- hydrant.swapSegment(null);
- // remove hydrant from persisted metadata:
- persistedHydrantMetadata.remove(hydrant);
+ partialSegment.swapSegment(null);
+ // remove partialSegment from persisted metadata:
+ persistedPartialSegmentMetadata.remove(partialSegment);
}
if (removeOnDiskData) {
@@ -1477,7 +1510,7 @@ private Committed readCommit() throws IOException
{
final File commitFile = computeCommitFile();
if (commitFile.exists()) {
- // merge current hydrants with existing hydrants
+ // merge current partial segments with existing partial segments
return objectMapper.readValue(commitFile, Committed.class);
} else {
return null;
@@ -1526,27 +1559,26 @@ private File createPersistDirIfNeeded(SegmentIdWithShardSpec identifier) throws
}
/**
- * Persists the given hydrant and returns the number of rows persisted. Must only be called in the single-threaded
- * persistExecutor.
- *
- * @param indexToPersist hydrant to persist
- * @param identifier the segment this hydrant is going to be part of
+ * Persists the given {@link PartialSegment} and returns the number of rows persisted. Must only be called in the
+ * single-threaded persistExecutor.
*
+ * @param indexToPersist {@link PartialSegment} to persist
+ * @param identifier the segment this {@link PartialSegment} is going to be part of
* @return the number of rows persisted
*/
- private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec identifier)
+ private int persistPartialSegment(PartialSegment indexToPersist, SegmentIdWithShardSpec identifier)
{
synchronized (indexToPersist) {
if (indexToPersist.hasSwapped()) {
log.info(
- "Segment[%s] hydrant[%s] already swapped. Ignoring request to persist.",
+ "Segment[%s] PartialSegment[%s] already swapped. Ignoring request to persist.",
identifier,
indexToPersist
);
return 0;
}
- log.debug("Segment[%s], persisting Hydrant[%s]", identifier, indexToPersist);
+ log.debug("Segment[%s], persisting PartialSegment[%s]", identifier, indexToPersist);
try {
final long startTime = System.nanoTime();
@@ -1576,7 +1608,7 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec id
segmentToSwap = new QueryableIndexSegment(indexIO.loadIndex(persistedFile), indexToPersist.getSegmentId());
} else {
// remember file path & segment id to rebuild the queryable index for merge:
- persistedHydrantMetadata.put(indexToPersist, new Pair<>(persistedFile, indexToPersist.getSegmentId()));
+ persistedPartialSegmentMetadata.put(indexToPersist, new Pair<>(persistedFile, indexToPersist.getSegmentId()));
}
indexToPersist.swapSegment(segmentToSwap);
@@ -1608,31 +1640,31 @@ private void removeDirectory(final File target)
}
}
- private int calculateMMappedHydrantMemoryInUsed(FireHydrant hydrant)
+ private int calculateMMappedPartialSegmentMemoryInUsed(PartialSegment partialSEgment)
{
if (skipBytesInMemoryOverheadCheck) {
return 0;
}
// These calculations are approximated from actual heap dumps.
- // Memory footprint includes count integer in FireHydrant, shorts in ReferenceCountingSegment,
+ // Memory footprint includes count integer in PartialSegment, shorts in ReferenceCountingSegment,
// Objects in SimpleQueryableIndex (such as SmooshedFileMapper, each ColumnHolder in column map, etc.)
int total;
- total = Integer.BYTES + (4 * Short.BYTES) + ROUGH_OVERHEAD_PER_HYDRANT;
+ total = Integer.BYTES + (4 * Short.BYTES) + ROUGH_OVERHEAD_PER_PARTIAL_SEGMENT;
if (isOpenSegments()) {
// for real time add references to byte memory mapped references..
- total += (hydrant.getSegmentNumDimensionColumns() * ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER) +
- (hydrant.getSegmentNumMetricColumns() * ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) +
+ total += (partialSEgment.getSegmentNumDimensionColumns() * ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER) +
+ (partialSEgment.getSegmentNumMetricColumns() * ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) +
ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER;
}
return total;
}
- private int calculateSinkMemoryInUsed()
+ private int calculateAppendableSegmentMemoryInUsed()
{
if (skipBytesInMemoryOverheadCheck) {
return 0;
}
- // Rough estimate of memory footprint of empty Sink based on actual heap dumps
- return ROUGH_OVERHEAD_PER_SINK;
+ // Rough estimate of memory footprint of empty AppendableSegment based on actual heap dumps
+ return ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT;
}
}
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java
index 56af5f3f62c9..871149008903 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java
@@ -74,7 +74,7 @@ public static Appenderator createRealtime(
dataSegmentPusher,
objectMapper,
segmentAnnouncer,
- new SinkQuerySegmentWalker(
+ new AppendableSegmentQuerySegmentWalker(
schema.getDataSource(),
new VersionedIntervalTimeline<>(
String.CASE_INSENSITIVE_ORDER
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java
index b544d33705cc..006b62ecdea2 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java
@@ -67,9 +67,9 @@
import org.apache.druid.segment.loading.DataSegmentPusher;
import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig;
import org.apache.druid.segment.metadata.FingerprintGenerator;
-import org.apache.druid.segment.realtime.FireHydrant;
+import org.apache.druid.segment.realtime.AppendableSegment;
+import org.apache.druid.segment.realtime.PartialSegment;
import org.apache.druid.segment.realtime.SegmentGenerationMetrics;
-import org.apache.druid.segment.realtime.sink.Sink;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;
@@ -96,16 +96,16 @@
* This is a new class produced when the old {@code AppenderatorImpl} was split. For historical
* reasons, the code for creating segments was all handled by the same code path in that class. The code
* was correct but inefficient for batch ingestion from a memory perspective. If the input file being processed
- * by batch ingestion had enough sinks & hydrants produced then it may run out of memory either in the
- * hydrant creation phase (append) of this class or in the merge hydrants phase. Therefore, a new class,
- * {@code BatchAppenderator}, this class, was created to specialize in batch ingestion and the old class
- * for stream ingestion was renamed to {@link StreamAppenderator}.
+ * by batch ingestion had enough {@link AppendableSegment} & {@link PartialSegment} produced then it may run
+ * out of memory either in the {@link PartialSegment} creation phase (append) of this class or in the merge
+ * {@link PartialSegment} phase. Therefore, a new class, {@code BatchAppenderator}, this class, was created to
+ * specialize in batch ingestion and the old class for stream ingestion was renamed to {@link StreamAppenderator}.
*/
public class BatchAppenderator implements Appenderator
{
- public static final int ROUGH_OVERHEAD_PER_SINK = 5000;
- // Rough estimate of memory footprint of empty FireHydrant based on actual heap dumps
- public static final int ROUGH_OVERHEAD_PER_HYDRANT = 1000;
+ public static final int ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT = 5000;
+ // Rough estimate of memory footprint of empty PartialSegment based on actual heap dumps
+ public static final int ROUGH_OVERHEAD_PER_PARTIAL_SEGMENT = 1000;
private static final EmittingLogger log = new EmittingLogger(BatchAppenderator.class);
private static final String IDENTIFIER_FILE_NAME = "identifier.json";
@@ -134,15 +134,16 @@ public class BatchAppenderator implements Appenderator
* then initalized just before scheduling the persit callable in the {@link #persistAll(Committer)}
* method, so no longer need to use a syncronized map.
*/
- private final Map sinks = new HashMap<>();
+ private final Map appendableSegments = new HashMap<>();
/**
- * The following sinks metadata map and associated class are the way to retain metadata now that sinks
- * are being completely removed from memory after each incremental persist. This map needs to be concurrent
- * since it is mutated in various methods potentially in different threads.
+ * The following appendable segments metadata map and associated class are the way to retain metadata now that
+ * appendable segments are being completely removed from memory after each incremental persist. This map needs to
+ * be concurrent since it is mutated in various methods potentially in different threads.
* For example mutated in {@link #add} when adding rows
- * and accessed/mutated in {@link #persistHydrant} during persists.
+ * and accessed/mutated in {@link #persistPartialSegment} during persists.
*/
- private final ConcurrentHashMap sinksMetadata = new ConcurrentHashMap<>();
+ private final ConcurrentHashMap appendableSegmentsMetadata =
+ new ConcurrentHashMap<>();
// This variable updated in add(), persist(), and drop()
private int rowsCurrentlyInMemory = 0;
@@ -286,29 +287,29 @@ public AppenderatorAddResult add(
);
}
- final Sink sink = getOrCreateSink(identifier);
+ final AppendableSegment appendableSegment = getOrCreateAppendableSegment(identifier);
metrics.reportMessageMaxTimestamp(row.getTimestampFromEpoch());
- final int sinkRowsInMemoryBeforeAdd = sink.getNumRowsInMemory();
- final int sinkRowsInMemoryAfterAdd;
- final long bytesInMemoryBeforeAdd = sink.getBytesInMemory();
+ final int appendableSegmentRowsInMemoryBeforeAdd = appendableSegment.getNumRowsInMemory();
+ final int appendableSegmentRowsInMemoryAfterAdd;
+ final long bytesInMemoryBeforeAdd = appendableSegment.getBytesInMemory();
final long bytesInMemoryAfterAdd;
final IncrementalIndexAddResult addResult;
try {
- addResult = sink.add(row, false); // allow incrememtal persis is always true for batch
- sinkRowsInMemoryAfterAdd = addResult.getRowCount();
+ addResult = appendableSegment.add(row, false); // allow incrememtal persis is always true for batch
+ appendableSegmentRowsInMemoryAfterAdd = addResult.getRowCount();
bytesInMemoryAfterAdd = addResult.getBytesInMemory();
}
catch (IndexSizeExceededException e) {
// Uh oh, we can't do anything about this! We can't persist (commit metadata would be out of sync) and we
// can't add the row (it just failed). This should never actually happen, though, because we check
- // sink.canAddRow after returning from add.
- log.error(e, "Sink for segment[%s] was unexpectedly full!", identifier);
+ // appendableSEgment.canAddRow after returning from add.
+ log.error(e, "Appendable segment[%s] was unexpectedly full!", identifier);
throw e;
}
- if (sinkRowsInMemoryAfterAdd < 0) {
- throw new SegmentNotWritableException("Attempt to add row to swapped-out sink for segment[%s].", identifier);
+ if (appendableSegmentRowsInMemoryAfterAdd < 0) {
+ throw new SegmentNotWritableException("Attempt to add row to swapped-out appendable segment[%s].", identifier);
}
if (addResult.isRowAdded()) {
@@ -317,18 +318,18 @@ public AppenderatorAddResult add(
parseExceptionHandler.handle(addResult.getParseException());
}
- final int numAddedRows = sinkRowsInMemoryAfterAdd - sinkRowsInMemoryBeforeAdd;
+ final int numAddedRows = appendableSegmentRowsInMemoryAfterAdd - appendableSegmentRowsInMemoryBeforeAdd;
rowsCurrentlyInMemory += numAddedRows;
bytesCurrentlyInMemory += (bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd);
totalRows += numAddedRows;
- sinksMetadata.computeIfAbsent(identifier, unused -> new SinkMetadata()).addRows(numAddedRows);
+ appendableSegmentsMetadata.computeIfAbsent(identifier, unused -> new AppendableSegmentMetadata()).addRows(numAddedRows);
boolean persist = false;
List persistReasons = new ArrayList<>();
- if (!sink.canAppendRow()) {
+ if (!appendableSegment.canAppendRow()) {
persist = true;
- persistReasons.add("No more rows can be appended to sink");
+ persistReasons.add("No more rows can be appended to appendable segment");
}
if (rowsCurrentlyInMemory >= tuningConfig.getMaxRowsInMemory()) {
persist = true;
@@ -352,13 +353,13 @@ public AppenderatorAddResult add(
log.info("Incremental persist to disk because %s.", String.join(",", persistReasons));
long bytesToBePersisted = 0L;
- for (Map.Entry entry : sinks.entrySet()) {
- final Sink sinkEntry = entry.getValue();
- if (sinkEntry != null) {
- bytesToBePersisted += sinkEntry.getBytesInMemory();
- if (sinkEntry.swappable()) {
- // Code for batch no longer memory maps hydrants, but they still take memory...
- int memoryStillInUse = calculateMemoryUsedByHydrant();
+ for (Map.Entry entry : appendableSegments.entrySet()) {
+ final AppendableSegment appendableSegmentEntry = entry.getValue();
+ if (appendableSegmentEntry != null) {
+ bytesToBePersisted += appendableSegmentEntry.getBytesInMemory();
+ if (appendableSegmentEntry.swappable()) {
+ // Code for batch no longer memory maps partial segments, but they still take memory...
+ int memoryStillInUse = calculateMemoryUsedByPartialSegments();
bytesCurrentlyInMemory += memoryStillInUse;
}
}
@@ -370,10 +371,10 @@ public AppenderatorAddResult add(
// This means that we ran out of all available memory to ingest (due to overheads created as part of ingestion)
final String alertMessage = StringUtils.format(
"Task has exceeded safe estimated heap usage limits, failing "
- + "(numSinks: [%d] numHydrantsAcrossAllSinks: [%d] totalRows: [%d])"
+ + "(numAppendableSegments: [%d] numPartialSegmentsAcrossAllAppendableSegments: [%d] totalRows: [%d])"
+ "(bytesCurrentlyInMemory: [%d] - bytesToBePersisted: [%d] > maxBytesTuningConfig: [%d])",
- sinks.size(),
- sinks.values().stream().mapToInt(Iterables::size).sum(),
+ appendableSegments.size(),
+ appendableSegments.values().stream().mapToInt(Iterables::size).sum(),
getTotalRowCount(),
bytesCurrentlyInMemory,
bytesToBePersisted,
@@ -414,7 +415,7 @@ public void onFailure(Throwable t)
MoreExecutors.directExecutor()
);
}
- return new AppenderatorAddResult(identifier, sinksMetadata.get(identifier).numRowsInSegment, false);
+ return new AppenderatorAddResult(identifier, appendableSegmentsMetadata.get(identifier).numRowsInSegment, false);
}
/**
@@ -423,19 +424,19 @@ public void onFailure(Throwable t)
@Override
public List getSegments()
{
- return ImmutableList.copyOf(sinksMetadata.keySet());
+ return ImmutableList.copyOf(appendableSegmentsMetadata.keySet());
}
@VisibleForTesting
public List getInMemorySegments()
{
- return ImmutableList.copyOf(sinks.keySet());
+ return ImmutableList.copyOf(appendableSegments.keySet());
}
@Override
public int getRowCount(final SegmentIdWithShardSpec identifier)
{
- return sinksMetadata.get(identifier).getNumRowsInSegment();
+ return appendableSegmentsMetadata.get(identifier).getNumRowsInSegment();
}
@Override
@@ -459,21 +460,21 @@ public long getBytesCurrentlyInMemory()
@VisibleForTesting
public long getBytesInMemory(SegmentIdWithShardSpec identifier)
{
- final Sink sink = sinks.get(identifier);
+ final AppendableSegment appendableSegment = appendableSegments.get(identifier);
- if (sink == null) {
- return 0L; // sinks are removed after a persist
+ if (appendableSegment == null) {
+ return 0L; // appendable segments are removed after a persist
} else {
- return sink.getBytesInMemory();
+ return appendableSegment.getBytesInMemory();
}
}
- private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier)
+ private AppendableSegment getOrCreateAppendableSegment(final SegmentIdWithShardSpec identifier)
{
- Sink retVal = sinks.get(identifier);
+ AppendableSegment retVal = appendableSegments.get(identifier);
if (retVal == null) {
- retVal = new Sink(
+ retVal = new AppendableSegment(
identifier.getInterval(),
schema,
identifier.getShardSpec(),
@@ -483,9 +484,9 @@ private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier)
maxBytesTuningConfig,
useMaxMemoryEstimates
);
- bytesCurrentlyInMemory += calculateSinkMemoryInUsed();
- sinks.put(identifier, retVal);
- metrics.setSinkCount(sinks.size());
+ bytesCurrentlyInMemory += calculateAppendableSegmentMemoryInUsed();
+ appendableSegments.put(identifier, retVal);
+ metrics.setAppendableSegmentCount(appendableSegments.size());
}
return retVal;
@@ -507,28 +508,32 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final
public void clear()
{
throwPersistErrorIfExists();
- clear(sinks, true);
+ clear(appendableSegments, true);
}
- private void clear(Map sinksToClear, boolean removeOnDiskData)
+ private void clear(Map appendableSegmentsToClear, boolean removeOnDiskData)
{
// Drop commit metadata, then abandon all segments.
- log.info("Clearing all[%d] sinks & their hydrants, removing data on disk: [%s]",
- sinksToClear.size(), removeOnDiskData);
+ log.info(
+ "Clearing all[%d] appendable segments & their partial segments, removing data on disk: [%s]",
+ appendableSegmentsToClear.size(),
+ removeOnDiskData
+ );
// Drop everything.
- Iterator> sinksIterator = sinksToClear.entrySet().iterator();
- sinksIterator.forEachRemaining(entry -> {
- clearSinkMemoryCountersAndDiskStoredData(entry.getKey(), entry.getValue(), removeOnDiskData);
- sinksIterator.remove();
+ Iterator> appendableSegmentsIterator =
+ appendableSegmentsToClear.entrySet().iterator();
+ appendableSegmentsIterator.forEachRemaining(entry -> {
+ clearAppendableSegmentsMemoryCountersAndDiskStoredData(entry.getKey(), entry.getValue(), removeOnDiskData);
+ appendableSegmentsIterator.remove();
});
- metrics.setSinkCount(sinksToClear.size());
+ metrics.setAppendableSegmentCount(appendableSegmentsToClear.size());
}
@Override
public ListenableFuture> drop(final SegmentIdWithShardSpec identifier)
{
- final Sink sink = sinks.get(identifier);
- SinkMetadata sm = sinksMetadata.remove(identifier);
+ final AppendableSegment appendableSegment = appendableSegments.get(identifier);
+ AppendableSegmentMetadata sm = appendableSegmentsMetadata.remove(identifier);
if (sm != null) {
int originalTotalRows = getTotalRowCount();
int rowsToDrop = sm.getNumRowsInSegment();
@@ -538,10 +543,10 @@ public ListenableFuture> drop(final SegmentIdWithShardSpec identifier)
}
totalRows = Math.max(totalRowsAfter, 0);
}
- if (sink != null) {
- clearSinkMemoryCountersAndDiskStoredData(identifier, sink, true);
- if (sinks.remove(identifier) == null) {
- log.warn("Sink for identifier[%s] not found, skipping", identifier);
+ if (appendableSegment != null) {
+ clearAppendableSegmentsMemoryCountersAndDiskStoredData(identifier, appendableSegment, true);
+ if (appendableSegments.remove(identifier) == null) {
+ log.warn("Appendable segment[%s] not found, skipping", identifier);
}
}
return Futures.immediateFuture(null);
@@ -556,44 +561,47 @@ public ListenableFuture persistAll(@Nullable final Committer committer)
if (committer != null) {
throw new ISE("committer must be null for BatchAppenderator");
}
- // Get ready to persist all sinks:
- final Map sinksToPersist = swapSinks();
+ // Get ready to persist all appendable segments:
+ final Map appendableSegmentsToPersist = swapAppendableSegments();
final Stopwatch runExecStopwatch = Stopwatch.createStarted();
ListenableFuture future = persistExecutor.submit(
() -> {
log.info("Spawning intermediate persist");
- // figure out hydrants (indices) to persist:
- final List> indexesToPersist = new ArrayList<>();
+ // figure out partial segments to persist:
+ final List> indexesToPersist = new ArrayList<>();
int numPersistedRows = 0;
long bytesPersisted = 0;
- int totalHydrantsCount = 0;
- final long totalSinks = sinksToPersist.size();
- for (Map.Entry entry : sinksToPersist.entrySet()) {
+ int totalPartialSegmentsCount = 0;
+ final long totalAppendableSegments = appendableSegmentsToPersist.size();
+ for (Map.Entry entry : appendableSegmentsToPersist.entrySet()) {
final SegmentIdWithShardSpec identifier = entry.getKey();
- final Sink sink = entry.getValue();
- if (sink == null) {
- throw new ISE("No sink for identifier: %s", identifier);
+ final AppendableSegment appendableSegment = entry.getValue();
+ if (appendableSegment == null) {
+ throw new ISE("No appendable segment: %s", identifier);
}
- final List hydrants = Lists.newArrayList(sink);
- // Since everytime we persist we also get rid of the in-memory references to sink & hydrants
- // the invariant of exactly one, always swappable, sink with exactly one unpersisted hydrant must hold
- int totalHydrantsForSink = hydrants.size();
- if (totalHydrantsForSink != 1) {
- throw new ISE("There should be only one hydrant for identifier[%s] but there are[%s]",
- identifier, totalHydrantsForSink
+ final List partialSegments = Lists.newArrayList(appendableSegment);
+ // Since everytime we persist we also get rid of the in-memory references to appendable segment & partial
+ // segment the invariant of exactly one, always swappable, appendable segment with exactly one
+ // unpersisted partial segment must hold
+ int totalPartialSegmentsForAppendableSegment = partialSegments.size();
+ if (totalPartialSegmentsForAppendableSegment != 1) {
+ throw new ISE(
+ "There should be only one partial segment for identifier[%s] but there are[%s]",
+ identifier,
+ totalPartialSegmentsForAppendableSegment
);
}
- totalHydrantsCount++;
- numPersistedRows += sink.getNumRowsInMemory();
- bytesPersisted += sink.getBytesInMemory();
+ totalPartialSegmentsCount++;
+ numPersistedRows += appendableSegment.getNumRowsInMemory();
+ bytesPersisted += appendableSegment.getBytesInMemory();
- if (!sink.swappable()) {
- throw new ISE("Sink is not swappable![%s]", identifier);
+ if (!appendableSegment.swappable()) {
+ throw new ISE("Appendable segment is not swappable![%s]", identifier);
}
- indexesToPersist.add(Pair.of(sink.swap(), identifier));
+ indexesToPersist.add(Pair.of(appendableSegment.swap(), identifier));
}
@@ -602,8 +610,8 @@ public ListenableFuture persistAll(@Nullable final Committer committer)
}
final Stopwatch persistStopwatch = Stopwatch.createStarted();
try {
- for (Pair pair : indexesToPersist) {
- metrics.incrementRowOutputCount(persistHydrant(pair.lhs, pair.rhs));
+ for (Pair pair : indexesToPersist) {
+ metrics.incrementRowOutputCount(persistPartialSegment(pair.lhs, pair.rhs));
}
log.info(
@@ -615,14 +623,14 @@ public ListenableFuture persistAll(@Nullable final Committer committer)
.collect(Collectors.joining(", "))
);
log.info(
- "Persisted stats: processed rows: [%d], persisted rows[%d], persisted sinks: [%d], persisted fireHydrants (across sinks): [%d]",
+ "Persisted stats: processed rows: [%d], persisted rows[%d], persisted appendable segments: [%d], persisted partial segment (across appendable segments): [%d]",
rowIngestionMeters.getProcessed(),
numPersistedRows,
- totalSinks,
- totalHydrantsCount
+ totalAppendableSegments,
+ totalPartialSegmentsCount
);
- // note that we do not need to reset sinks metadata since we did it at the start...
+ // note that we do not need to reset appendable segments metadata since we did it at the start...
}
catch (Exception e) {
@@ -635,8 +643,11 @@ public ListenableFuture persistAll(@Nullable final Committer committer)
metrics.incrementPersistTimeMillis(persistMillis);
persistStopwatch.stop();
// make sure no push can start while persisting:
- log.info("Persisted rows[%,d] and bytes[%,d] and removed all sinks & hydrants from memory in[%d] millis",
- numPersistedRows, bytesPersisted, persistMillis
+ log.info(
+ "Persisted rows[%,d] and bytes[%,d] and removed all appendable segments & partial segments from memory in[%d] millis",
+ numPersistedRows,
+ bytesPersisted,
+ persistMillis
);
log.info("Persist is done.");
}
@@ -654,16 +665,16 @@ public ListenableFuture persistAll(@Nullable final Committer committer)
}
/**
- * All sinks will be persisted so do a shallow copy of the Sinks map, reset
+ * All appendable segments will be persisted so do a shallow copy of the {@link #appendableSegments} map, reset
* the map and metadata (i.e. memory consumption counters) so that ingestion can go on
- * @return The map of sinks to persist, this map will be garbage collected after
+ * @return The map of appendable segments to persist, this map will be garbage collected after
* persist is complete since we will not be keeping a reference to it...
*/
- Map swapSinks()
+ Map swapAppendableSegments()
{
- Map retVal = ImmutableMap.copyOf(sinks);
- sinks.clear();
- resetSinkMetadata();
+ Map retVal = ImmutableMap.copyOf(appendableSegments);
+ appendableSegments.clear();
+ resetAppendableSegmentMetadata();
return retVal;
}
@@ -690,33 +701,33 @@ public ListenableFuture push(
persistAll(null), // make sure persists is done before push...
(Function) commitMetadata -> {
- log.info("Push started, processsing[%d] sinks", identifiers.size());
+ log.info("Push started, processsing[%d] appendable segments", identifiers.size());
- int totalHydrantsMerged = 0;
+ int totalPartialSegmentsMerged = 0;
for (SegmentIdWithShardSpec identifier : identifiers) {
- SinkMetadata sm = sinksMetadata.get(identifier);
+ AppendableSegmentMetadata sm = appendableSegmentsMetadata.get(identifier);
if (sm == null) {
- throw new ISE("No sink has been processed for identifier[%s]", identifier);
+ throw new ISE("No appendable segment has been processed for identifier[%s]", identifier);
}
File persistedDir = sm.getPersistedFileDir();
if (persistedDir == null) {
- throw new ISE("Persisted directory for identifier[%s] is null in sink metadata", identifier);
+ throw new ISE("Persisted directory for identifier[%s] is null in appendable segment metadata", identifier);
}
- totalHydrantsMerged += sm.getNumHydrants();
+ totalPartialSegmentsMerged += sm.getNumPartialSegments();
- // retrieve sink from disk:
- Sink sinkForIdentifier;
+ // retrieve appendable segment from disk:
+ AppendableSegment appendableSegmentForIdentifier;
try {
- sinkForIdentifier = getSinkForIdentifierPath(identifier, persistedDir);
+ appendableSegmentForIdentifier = getAppendableSegmentForIdentifierPath(identifier, persistedDir);
}
catch (IOException e) {
- throw new ISE(e, "Failed to retrieve sinks for identifier[%s]", identifier);
+ throw new ISE(e, "Failed to retrieve appendable segment for identifier[%s]", identifier);
}
// push it:
final DataSegmentWithMetadata dataSegmentWithMetadata = mergeAndPush(
identifier,
- sinkForIdentifier
+ appendableSegmentForIdentifier
);
// record it:
@@ -740,8 +751,8 @@ public ListenableFuture push(
log.warn("mergeAndPush[%s] returned null, skipping.", identifier);
}
}
- log.info("Push done: total sinks merged[%d], total hydrants merged[%d]",
- identifiers.size(), totalHydrantsMerged
+ log.info("Push done: total appendable segments merged[%d], total partial segments merged[%d]",
+ identifiers.size(), totalPartialSegmentsMerged
);
return new SegmentsAndCommitMetadata(dataSegments, commitMetadata, segmentSchemaMapping);
},
@@ -753,13 +764,13 @@ public ListenableFuture push(
/**
* Merge segment, push to deep storage. Should only be used on segments that have been fully persisted.
*
- * @param identifier sink identifier
- * @param sink sink to push
- * @return segment descriptor along with schema, or null if the sink is no longer valid
+ * @param identifier appendable segment identifier
+ * @param appendableSegment appendable segment to push
+ * @return segment descriptor along with schema, or null if the appendable segment is no longer valid
*/
private DataSegmentWithMetadata mergeAndPush(
final SegmentIdWithShardSpec identifier,
- final Sink sink
+ final AppendableSegment appendableSegment
)
{
// Use a descriptor file to indicate that pushing has completed.
@@ -768,24 +779,28 @@ private DataSegmentWithMetadata mergeAndPush(
final File descriptorFile = computeDescriptorFile(identifier);
// Sanity checks
- if (sink.isWritable()) {
- throw new ISE("Expected sink to be no longer writable before mergeAndPush for segment[%s].", identifier);
+ if (appendableSegment.isWritable()) {
+ throw new ISE("Expected appendable segment to be no longer writable before mergeAndPush for segment[%s].", identifier);
}
- int numHydrants = 0;
- for (FireHydrant hydrant : sink) {
- if (!hydrant.hasSwapped()) {
- throw new ISE("Expected sink to be fully persisted before mergeAndPush for segment[%s].", identifier);
+ int numPartialSegments = 0;
+ for (PartialSegment partialSegment : appendableSegment) {
+ if (!partialSegment.hasSwapped()) {
+ throw new ISE("Expected appendable segment to be fully persisted before mergeAndPush for segment[%s].", identifier);
}
- numHydrants++;
+ numPartialSegments++;
}
- SinkMetadata sm = sinksMetadata.get(identifier);
+ AppendableSegmentMetadata sm = appendableSegmentsMetadata.get(identifier);
if (sm == null) {
- log.warn("Sink metadata not found just before merge for identifier [%s]", identifier);
- } else if (numHydrants != sm.getNumHydrants()) {
- throw new ISE("Number of restored hydrants[%d] for identifier[%s] does not match expected value[%d]",
- numHydrants, identifier, sm.getNumHydrants());
+ log.warn("Appendable segment metadata not found just before merge for identifier [%s]", identifier);
+ } else if (numPartialSegments != sm.getNumPartialSegments()) {
+ throw new ISE(
+ "Number of restored partial segment[%d] for identifier[%s] does not match expected value[%d]",
+ numPartialSegments,
+ identifier,
+ sm.getNumPartialSegments()
+ );
}
try {
@@ -814,13 +829,13 @@ private DataSegmentWithMetadata mergeAndPush(
long rowsinMergedSegment = 0L;
Closer closer = Closer.create();
try {
- for (FireHydrant fireHydrant : sink) {
- Pair segmentAndCloseable = fireHydrant.getAndIncrementSegment();
+ for (PartialSegment partialSegment : appendableSegment) {
+ Pair segmentAndCloseable = partialSegment.getAndIncrementSegment();
final QueryableIndex queryableIndex = segmentAndCloseable.lhs.asQueryableIndex();
if (queryableIndex != null) {
rowsinMergedSegment += queryableIndex.getNumRows();
}
- log.debug("Segment[%s] adding hydrant[%s]", identifier, fireHydrant);
+ log.debug("Segment[%s] adding partial segment[%s]", identifier, partialSegment);
indexes.add(queryableIndex);
closer.register(segmentAndCloseable.rhs);
}
@@ -853,8 +868,8 @@ private DataSegmentWithMetadata mergeAndPush(
// dataSegmentPusher retries internally when appropriate; no need for retries here.
final DataSegment segment = dataSegmentPusher.push(
mergedFile,
- sink.getSegment()
- .withDimensions(
+ appendableSegment.getSegment()
+ .withDimensions(
IndexMerger.getMergedDimensionsFromQueryableIndexes(
indexes,
schema.getDimensionsSpec()
@@ -863,11 +878,11 @@ private DataSegmentWithMetadata mergeAndPush(
false
);
- // Drop the queryable indexes behind the hydrants... they are not needed anymore and their
+ // Drop the queryable indexes behind the partial segment... they are not needed anymore and their
// mapped file references
// can generate OOMs during merge if enough of them are held back...
- for (FireHydrant fireHydrant : sink) {
- fireHydrant.swapSegment(null);
+ for (PartialSegment partialSegment : appendableSegment) {
+ partialSegment.swapSegment(null);
}
SchemaPayloadPlus schemaMetadata =
@@ -875,7 +890,7 @@ private DataSegmentWithMetadata mergeAndPush(
? TaskSegmentSchemaUtil.getSegmentSchema(mergedTarget, indexIO)
: null;
- // cleanup, sink no longer needed
+ // cleanup, appendable segment no longer needed
removeDirectory(computePersistDir(identifier));
final long pushFinishTime = System.nanoTime();
@@ -942,7 +957,7 @@ public void close()
log.debug("Waited for and shutdown executors...");
// cleanup:
- clear(sinks, false);
+ clear(appendableSegments, false);
unlockBasePersistDirectory();
List persistedIdentifiers = getPersistedidentifierPaths();
if (persistedIdentifiers != null) {
@@ -952,7 +967,7 @@ public void close()
}
totalRows = 0;
- sinksMetadata.clear();
+ appendableSegmentsMetadata.clear();
}
@Override
@@ -1020,52 +1035,52 @@ public List getPersistedidentifierPaths()
return null;
}
- for (File sinkDir : files) {
- final File identifierFile = new File(sinkDir, IDENTIFIER_FILE_NAME);
+ for (File appendableSegmentDir : files) {
+ final File identifierFile = new File(appendableSegmentDir, IDENTIFIER_FILE_NAME);
if (!identifierFile.isFile()) {
- // No identifier in this sinkDir; it must not actually be a sink directory. Skip it.
+ // No identifier in this appendableSegmentDir; it must not actually be a appendable segment directory. Skip it.
continue;
}
- retVal.add(sinkDir);
+ retVal.add(appendableSegmentDir);
}
return retVal;
}
- private Sink getSinkForIdentifierPath(SegmentIdWithShardSpec identifier, File identifierPath)
+ private AppendableSegment getAppendableSegmentForIdentifierPath(SegmentIdWithShardSpec identifier, File identifierPath)
throws IOException
{
// To avoid reading and listing of "merged" dir and other special files
- final File[] sinkFiles = identifierPath.listFiles(
+ final File[] appendableSegmentFiles = identifierPath.listFiles(
(dir, fileName) -> !(Ints.tryParse(fileName) == null)
);
- if (sinkFiles == null) {
- throw new ISE("Problem reading persisted sinks in path[%s]", identifierPath);
+ if (appendableSegmentFiles == null) {
+ throw new ISE("Problem reading persisted appendable segments in path[%s]", identifierPath);
}
Arrays.sort(
- sinkFiles,
+ appendableSegmentFiles,
(o1, o2) -> Ints.compare(Integer.parseInt(o1.getName()), Integer.parseInt(o2.getName()))
);
- List hydrants = new ArrayList<>();
- for (File hydrantDir : sinkFiles) {
- final int hydrantNumber = Integer.parseInt(hydrantDir.getName());
+ List partialSegments = new ArrayList<>();
+ for (File partialSegmentDir : appendableSegmentFiles) {
+ final int partialSegmentNumber = Integer.parseInt(partialSegmentDir.getName());
- log.debug("Loading previously persisted partial segment at [%s]", hydrantDir);
- if (hydrantNumber != hydrants.size()) {
- throw new ISE("Missing hydrant [%,d] in identifier [%s].", hydrants.size(), identifier);
+ log.debug("Loading previously persisted partial segment at [%s]", partialSegmentDir);
+ if (partialSegmentNumber != partialSegments.size()) {
+ throw new ISE("Missing partial segment [%,d] in identifier [%s].", partialSegments.size(), identifier);
}
- hydrants.add(
- new FireHydrant(
- new QueryableIndexSegment(indexIO.loadIndex(hydrantDir), identifier.asSegmentId()),
- hydrantNumber
+ partialSegments.add(
+ new PartialSegment(
+ new QueryableIndexSegment(indexIO.loadIndex(partialSegmentDir), identifier.asSegmentId()),
+ partialSegmentNumber
)
);
}
- Sink retVal = new Sink(
+ AppendableSegment retVal = new AppendableSegment(
identifier.getInterval(),
schema,
identifier.getShardSpec(),
@@ -1074,50 +1089,51 @@ private Sink getSinkForIdentifierPath(SegmentIdWithShardSpec identifier, File id
tuningConfig.getMaxRowsInMemory(),
maxBytesTuningConfig,
useMaxMemoryEstimates,
- hydrants
+ partialSegments
);
- retVal.finishWriting(); // this sink is not writable
+ retVal.finishWriting(); // this appendable segment is not writable
return retVal;
}
- private void resetSinkMetadata()
+ private void resetAppendableSegmentMetadata()
{
rowsCurrentlyInMemory = 0;
bytesCurrentlyInMemory = 0;
- metrics.setSinkCount(0);
+ metrics.setAppendableSegmentCount(0);
}
- // This function does not remove the sink from its tracking Map (sinks), the caller is responsible for that
- // this is because the Map is not synchronized and removing elements from a map while traversing it
- // throws concurrent access exception
- private void clearSinkMemoryCountersAndDiskStoredData(
+ // This function does not remove the appendable segment from its tracking Map (appendableSegments), the caller is
+ // responsible for that this is because the Map is not synchronized and removing elements from a map while
+ // traversing it throws concurrent access exception
+ private void clearAppendableSegmentsMemoryCountersAndDiskStoredData(
final SegmentIdWithShardSpec identifier,
- final Sink sink,
+ final AppendableSegment appendableSegment,
final boolean removeOnDiskData
)
{
- // Ensure no future writes will be made to this sink.
- if (sink.finishWriting()) {
- // Decrement this sink's rows from the counters. we only count active sinks so that we don't double decrement,
- // i.e. those that haven't been persisted for *InMemory counters, or pushed to deep storage for the total counter.
- rowsCurrentlyInMemory -= sink.getNumRowsInMemory();
- bytesCurrentlyInMemory -= sink.getBytesInMemory();
- bytesCurrentlyInMemory -= calculateSinkMemoryInUsed();
- for (FireHydrant hydrant : sink) {
- // Decrement memory used by all Memory Mapped Hydrant
- if (!hydrant.equals(sink.getCurrHydrant())) {
- bytesCurrentlyInMemory -= calculateMemoryUsedByHydrant();
+ // Ensure no future writes will be made to this appendable segment.
+ if (appendableSegment.finishWriting()) {
+ // Decrement this appendable segment's rows from the counters. we only count active appendable segments so that
+ // we don't double decrement, i.e. those that haven't been persisted for *InMemory counters, or pushed to deep
+ // storage for the total counter.
+ rowsCurrentlyInMemory -= appendableSegment.getNumRowsInMemory();
+ bytesCurrentlyInMemory -= appendableSegment.getBytesInMemory();
+ bytesCurrentlyInMemory -= calculateAppendableSegmentMemoryInUsed();
+ for (PartialSegment partialSegment : appendableSegment) {
+ // Decrement memory used by all Memory Mapped partial segment
+ if (!partialSegment.equals(appendableSegment.getCurrentPartialSegment())) {
+ bytesCurrentlyInMemory -= calculateMemoryUsedByPartialSegments();
}
}
- // totalRows are not decremented when removing the sink from memory, sink was just persisted, and it
- // still "lives" but it is in hibernation. It will be revived later just before push.
+ // totalRows are not decremented when removing the appendable segment from memory, appendable segment was just
+ // persisted, and it still "lives" but it is in hibernation. It will be revived later just before push.
}
if (removeOnDiskData) {
removeDirectory(computePersistDir(identifier));
}
- log.info("Removed sink for segment[%s].", identifier);
+ log.info("Removed appendable segment[%s].", identifier);
}
@@ -1152,40 +1168,40 @@ private File createPersistDirIfNeeded(SegmentIdWithShardSpec identifier) throws
}
/**
- * Persists the given hydrant and returns the number of rows persisted.
+ * Persists the given partial segment and returns the number of rows persisted.
*
- * @param indexToPersist hydrant to persist
- * @param identifier the segment this hydrant is going to be part of
+ * @param indexToPersist partial segment to persist
+ * @param identifier the segment this partial segment is going to be part of
* @return the number of rows persisted
*/
- private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec identifier)
+ private int persistPartialSegment(PartialSegment indexToPersist, SegmentIdWithShardSpec identifier)
{
if (indexToPersist.hasSwapped()) {
throw new ISE(
- "Segment[%s] hydrant[%s] already swapped. This cannot happen.",
+ "Segment[%s] partial segment[%s] already swapped. This cannot happen.",
identifier,
indexToPersist
);
}
- log.debug("Segment[%s], persisting Hydrant[%s]", identifier, indexToPersist);
+ log.debug("Segment[%s], persisting partial segment[%s]", identifier, indexToPersist);
try {
final long startTime = System.nanoTime();
int numRows = indexToPersist.getIndex().size();
- // since the sink may have been persisted before it may have lost its
- // hydrant count, we remember that value in the sinks' metadata, so we have
+ // since the appendable segment may have been persisted before it may have lost its
+ // partial segment count, we remember that value in the appendable segments' metadata, so we have
// to pull it from there....
- SinkMetadata sm = sinksMetadata.get(identifier);
+ AppendableSegmentMetadata sm = appendableSegmentsMetadata.get(identifier);
if (sm == null) {
- throw new ISE("Sink must not be null for identifier when persisting hydrant[%s]", identifier);
+ throw new ISE("Appendable segment must not be null for identifier when persisting partial segment[%s]", identifier);
}
final File persistDir = createPersistDirIfNeeded(identifier);
indexMerger.persist(
indexToPersist.getIndex(),
identifier.getInterval(),
- new File(persistDir, String.valueOf(sm.getNumHydrants())),
+ new File(persistDir, String.valueOf(sm.getNumPartialSegments())),
tuningConfig.getIndexSpecForIntermediatePersists(),
tuningConfig.getSegmentWriteOutMediumFactory()
);
@@ -1194,14 +1210,14 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec id
log.info(
"Persisted in-memory data for segment[%s] spill[%s] to disk in [%,d] ms (%,d rows).",
indexToPersist.getSegmentId(),
- sm.getNumHydrants(),
+ sm.getNumPartialSegments(),
(System.nanoTime() - startTime) / 1000000,
numRows
);
indexToPersist.swapSegment(null);
- // remember hydrant count:
- sm.addHydrants(1);
+ // remember partial segment count:
+ sm.addPartialSegments(1);
return numRows;
}
@@ -1231,56 +1247,56 @@ private void removeDirectory(final File target)
}
}
- private int calculateMemoryUsedByHydrant()
+ private int calculateMemoryUsedByPartialSegments()
{
if (skipBytesInMemoryOverheadCheck) {
return 0;
}
// These calculations are approximated from actual heap dumps.
int total;
- total = Integer.BYTES + (4 * Short.BYTES) + ROUGH_OVERHEAD_PER_HYDRANT;
+ total = Integer.BYTES + (4 * Short.BYTES) + ROUGH_OVERHEAD_PER_PARTIAL_SEGMENT;
return total;
}
- private int calculateSinkMemoryInUsed()
+ private int calculateAppendableSegmentMemoryInUsed()
{
if (skipBytesInMemoryOverheadCheck) {
return 0;
}
- // Rough estimate of memory footprint of empty Sink based on actual heap dumps
- return ROUGH_OVERHEAD_PER_SINK;
+ // Rough estimate of memory footprint of empty appendable segment based on actual heap dumps
+ return ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT;
}
/**
- * This class is used for information that needs to be kept related to Sinks as
+ * This class is used for information that needs to be kept related to {@link AppendableSegment} as
* they are persisted and removed from memory at every incremental persist.
* The information is used for sanity checks and as information required
* for functionality, depending on the field that is used. More info about the
* fields is annotated as comments in the class
*/
- private static class SinkMetadata
+ private static class AppendableSegmentMetadata
{
- /** This is used to maintain the rows in the sink accross persists of the sink
+ /** This is used to maintain the rows in the appendable segment accross persists of the partial segments
* used for functionality (i.e. to detect whether an incremental push
* is needed {@link AppenderatorDriverAddResult#isPushRequired(Integer, Long)}
**/
private int numRowsInSegment;
- /** For sanity check as well as functionality: to make sure that all hydrants for a sink are restored from disk at
- * push time and also to remember the fire hydrant "count" when persisting it.
+ /** For sanity check as well as functionality: to make sure that all partial segments for an appendable segment are
+ * restored from disk at push time and also to remember the partial segment "count" when persisting it.
*/
- private int numHydrants;
+ private int numPartialSegments;
/* Reference to directory that holds the persisted data */
File persistedFileDir;
- public SinkMetadata()
+ public AppendableSegmentMetadata()
{
this(0, 0);
}
- public SinkMetadata(int numRowsInSegment, int numHydrants)
+ public AppendableSegmentMetadata(int numRowsInSegment, int numPartialSegments)
{
this.numRowsInSegment = numRowsInSegment;
- this.numHydrants = numHydrants;
+ this.numPartialSegments = numPartialSegments;
}
public void addRows(int num)
@@ -1288,9 +1304,9 @@ public void addRows(int num)
numRowsInSegment += num;
}
- public void addHydrants(int num)
+ public void addPartialSegments(int num)
{
- numHydrants += num;
+ numPartialSegments += num;
}
public int getNumRowsInSegment()
@@ -1298,9 +1314,9 @@ public int getNumRowsInSegment()
return numRowsInSegment;
}
- public int getNumHydrants()
+ public int getNumPartialSegments()
{
- return numHydrants;
+ return numPartialSegments;
}
public void setPersistedFileDir(File persistedFileDir)
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Committed.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Committed.java
index ecedd48a506f..ee3f44f3a66d 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Committed.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Committed.java
@@ -31,36 +31,36 @@ public class Committed
{
private static final Committed NIL = new Committed(ImmutableMap.of(), null);
- // Map of segment identifierAsString -> number of committed hydrants
- private final ImmutableMap hydrants;
+ // Map of segment identifierAsString -> number of committed PartialSegment
+ private final ImmutableMap partialSegments;
private final Object metadata;
@JsonCreator
public Committed(
- @JsonProperty("hydrants") Map hydrants,
+ @JsonProperty("hydrants") Map partialSegments,
@JsonProperty("metadata") Object metadata
)
{
- this.hydrants = ImmutableMap.copyOf(hydrants);
+ this.partialSegments = ImmutableMap.copyOf(partialSegments);
this.metadata = metadata;
}
public static Committed create(
- Map hydrants0,
+ Map newPartialSegments,
Object metadata
)
{
- final ImmutableMap.Builder hydrants = ImmutableMap.builder();
- for (Map.Entry entry : hydrants0.entrySet()) {
- hydrants.put(entry.getKey().toString(), entry.getValue());
+ final ImmutableMap.Builder partialSegments = ImmutableMap.builder();
+ for (Map.Entry entry : newPartialSegments.entrySet()) {
+ partialSegments.put(entry.getKey().toString(), entry.getValue());
}
- return new Committed(hydrants.build(), metadata);
+ return new Committed(partialSegments.build(), metadata);
}
@JsonProperty
- public ImmutableMap getHydrants()
+ public ImmutableMap getPartialSegments()
{
- return hydrants;
+ return partialSegments;
}
@JsonProperty
@@ -69,25 +69,25 @@ public Object getMetadata()
return metadata;
}
- public int getCommittedHydrants(final String identifierAsString)
+ public int getCommittedPartialSegments(final String identifierAsString)
{
- final Integer committedHydrant = hydrants.get(identifierAsString);
- return committedHydrant == null ? 0 : committedHydrant;
+ final Integer committedPartialSegment = partialSegments.get(identifierAsString);
+ return committedPartialSegment == null ? 0 : committedPartialSegment;
}
public Committed without(final String identifierAsString)
{
- final Map newHydrants = new HashMap<>(hydrants);
- newHydrants.remove(identifierAsString);
- return new Committed(newHydrants, metadata);
+ final Map newPartialSegments = new HashMap<>(partialSegments);
+ newPartialSegments.remove(identifierAsString);
+ return new Committed(newPartialSegments, metadata);
}
- public Committed with(final Map hydrantsToAdd)
+ public Committed with(final Map partialSegmentsToAdd)
{
- final Map newHydrants = new HashMap<>();
- newHydrants.putAll(hydrants);
- newHydrants.putAll(hydrantsToAdd);
- return new Committed(newHydrants, metadata);
+ final Map newPartialSegments = new HashMap<>();
+ newPartialSegments.putAll(partialSegments);
+ newPartialSegments.putAll(partialSegmentsToAdd);
+ return new Committed(newPartialSegments, metadata);
}
@Override
@@ -100,21 +100,21 @@ public boolean equals(Object o)
return false;
}
Committed committed = (Committed) o;
- return Objects.equals(hydrants, committed.hydrants) &&
+ return Objects.equals(partialSegments, committed.partialSegments) &&
Objects.equals(metadata, committed.metadata);
}
@Override
public int hashCode()
{
- return Objects.hash(hydrants, metadata);
+ return Objects.hash(partialSegments, metadata);
}
@Override
public String toString()
{
return "Committed{" +
- "hydrants=" + hydrants +
+ "hydrants=" + partialSegments +
", metadata=" + metadata +
'}';
}
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentSchemas.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentSchemas.java
index 359da9bed76d..22ebf51a08f4 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentSchemas.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentSchemas.java
@@ -22,6 +22,7 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.realtime.AppendableSegment;
import java.util.List;
import java.util.Map;
@@ -30,7 +31,7 @@
/**
* Encapsulates schema information for multiple segments.
*
- * Primarily used to announce schema changes for all {@link org.apache.druid.segment.realtime.sink.Sink}
+ * Primarily used to announce schema changes for all {@link AppendableSegment}
* created by a task in {@link StreamAppenderator}.
*/
public class SegmentSchemas
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java
index 4d1253591e08..6929383078fb 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java
@@ -76,9 +76,9 @@
import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig;
import org.apache.druid.segment.metadata.FingerprintGenerator;
-import org.apache.druid.segment.realtime.FireHydrant;
+import org.apache.druid.segment.realtime.AppendableSegment;
+import org.apache.druid.segment.realtime.PartialSegment;
import org.apache.druid.segment.realtime.SegmentGenerationMetrics;
-import org.apache.druid.segment.realtime.sink.Sink;
import org.apache.druid.server.coordination.DataSegmentAnnouncer;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
@@ -122,10 +122,10 @@ public class StreamAppenderator implements Appenderator
public static final int ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER = 1000;
public static final int ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER = 700;
public static final int ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER = 600;
- // Rough estimate of memory footprint of empty Sink based on actual heap dumps
- public static final int ROUGH_OVERHEAD_PER_SINK = 5000;
- // Rough estimate of memory footprint of empty FireHydrant based on actual heap dumps
- public static final int ROUGH_OVERHEAD_PER_HYDRANT = 1000;
+ // Rough estimate of memory footprint of empty AppendableSegment based on actual heap dumps
+ public static final int ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT = 5000;
+ // Rough estimate of memory footprint of empty PartialSegment based on actual heap dumps
+ public static final int ROUGH_OVERHEAD_PER_PARTIAL_SEGMENT = 1000;
private static final EmittingLogger log = new EmittingLogger(StreamAppenderator.class);
private static final int WARN_DELAY = 1000;
@@ -147,10 +147,10 @@ public class StreamAppenderator implements Appenderator
* called) and from {@link #persistExecutor}. It could also be accessed (but not mutated) potentially in the context
* of any thread from {@link #drop}.
*/
- private final ConcurrentMap sinks = new ConcurrentHashMap<>();
+ private final ConcurrentMap appendableSegments = new ConcurrentHashMap<>();
private final ConcurrentMap idToPendingSegment = new ConcurrentHashMap<>();
- private final Set droppingSinks = Sets.newConcurrentHashSet();
- private final VersionedIntervalTimeline sinkTimeline;
+ private final Set droppingAppendableSegments = Sets.newConcurrentHashSet();
+ private final VersionedIntervalTimeline appendableSegmentTimeline;
private final long maxBytesTuningConfig;
private final boolean skipBytesInMemoryOverheadCheck;
private final boolean useMaxMemoryEstimates;
@@ -169,9 +169,9 @@ public class StreamAppenderator implements Appenderator
private final AtomicBoolean closed = new AtomicBoolean(false);
/**
- * Map from base segment identifier of a sink to the set of all the segment ids associated with it.
+ * Map from base segment identifier of a appendable segment to the set of all the segment ids associated with it.
* The set contains the base segment itself and its upgraded versions announced as a result of a concurrent replace.
- * The map contains all the available sinks' identifiers in its keyset.
+ * The map contains all the available appendable segment' identifiers in its keyset.
*/
private final ConcurrentMap> baseSegmentToUpgradedSegments
= new ConcurrentHashMap<>();
@@ -182,13 +182,14 @@ public class StreamAppenderator implements Appenderator
= new ConcurrentHashMap<>();
/**
* Set of all segment identifiers that have been marked to be abandoned.
- * This is used to determine if all the segments corresponding to a sink have been abandoned and it can be dropped.
+ * This is used to determine if all the segments corresponding to an appendable segment have been abandoned and it
+ * can be dropped.
*/
private final ConcurrentHashMap.KeySetView abandonedSegments
= ConcurrentHashMap.newKeySet();
- private final SinkSchemaAnnouncer sinkSchemaAnnouncer;
+ private final AppendableSegmentSchemaAnnouncer appendableSegmentSchemaAnnouncer;
private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig;
private volatile ListeningExecutorService persistExecutor = null;
@@ -208,12 +209,13 @@ public class StreamAppenderator implements Appenderator
private final FingerprintGenerator fingerprintGenerator;
/**
- * This constructor allows the caller to provide its own SinkQuerySegmentWalker.
+ * This constructor allows the caller to provide its own {@link AppendableSegmentQuerySegmentWalker}.
*
- * The sinkTimeline is set to the sink timeline of the provided SinkQuerySegmentWalker.
- * If the SinkQuerySegmentWalker is null, a new sink timeline is initialized.
+ * The {@link #appendableSegmentTimeline} is set to the appendable segment timeline of the provided
+ * {@link AppendableSegmentQuerySegmentWalker}. If the {@link AppendableSegmentQuerySegmentWalker} is null, a
+ * new timeline is initialized.
*
- * It is used by UnifiedIndexerAppenderatorsManager which allows queries on data associated with multiple
+ * It is used by {@link UnifiedIndexerAppenderatorsManager} which allows queries on data associated with multiple
* Appenderators.
*/
StreamAppenderator(
@@ -225,7 +227,7 @@ public class StreamAppenderator implements Appenderator
DataSegmentPusher dataSegmentPusher,
ObjectMapper objectMapper,
DataSegmentAnnouncer segmentAnnouncer,
- @Nullable SinkQuerySegmentWalker sinkQuerySegmentWalker,
+ @Nullable AppendableSegmentQuerySegmentWalker appendableSegmentQuerySegmentWalker,
IndexIO indexIO,
IndexMerger indexMerger,
Cache cache,
@@ -246,23 +248,23 @@ public class StreamAppenderator implements Appenderator
this.indexIO = Preconditions.checkNotNull(indexIO, "indexIO");
this.indexMerger = Preconditions.checkNotNull(indexMerger, "indexMerger");
this.cache = cache;
- this.texasRanger = sinkQuerySegmentWalker;
+ this.texasRanger = appendableSegmentQuerySegmentWalker;
this.rowIngestionMeters = Preconditions.checkNotNull(rowIngestionMeters, "rowIngestionMeters");
this.parseExceptionHandler = Preconditions.checkNotNull(parseExceptionHandler, "parseExceptionHandler");
- if (sinkQuerySegmentWalker == null) {
- this.sinkTimeline = new VersionedIntervalTimeline<>(
+ if (appendableSegmentQuerySegmentWalker == null) {
+ this.appendableSegmentTimeline = new VersionedIntervalTimeline<>(
String.CASE_INSENSITIVE_ORDER
);
} else {
- this.sinkTimeline = sinkQuerySegmentWalker.getSinkTimeline();
+ this.appendableSegmentTimeline = appendableSegmentQuerySegmentWalker.getAppendableSegmentTimeline();
}
maxBytesTuningConfig = tuningConfig.getMaxBytesInMemoryOrDefault();
skipBytesInMemoryOverheadCheck = tuningConfig.isSkipBytesInMemoryOverheadCheck();
this.useMaxMemoryEstimates = useMaxMemoryEstimates;
this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig;
- this.sinkSchemaAnnouncer = new SinkSchemaAnnouncer();
+ this.appendableSegmentSchemaAnnouncer = new AppendableSegmentSchemaAnnouncer();
this.exec = Executors.newScheduledThreadPool(
1,
@@ -296,10 +298,10 @@ public String getDataSource()
public Object startJob()
{
lockBasePersistDirectory();
- final Object retVal = bootstrapSinksFromDisk();
+ final Object retVal = bootstrapAppendableSegmentsFromDisk();
initializeExecutors();
resetNextFlush();
- sinkSchemaAnnouncer.start();
+ appendableSegmentSchemaAnnouncer.start();
return retVal;
}
@@ -328,29 +330,29 @@ public AppenderatorAddResult add(
);
}
- final Sink sink = getOrCreateSink(identifier);
+ final AppendableSegment appendableSegment = getOrCreateAppendableSegment(identifier);
metrics.reportMessageMaxTimestamp(row.getTimestampFromEpoch());
- final int sinkRowsInMemoryBeforeAdd = sink.getNumRowsInMemory();
- final int sinkRowsInMemoryAfterAdd;
- final long bytesInMemoryBeforeAdd = sink.getBytesInMemory();
+ final int appendableSegmentRowsInMemoryBeforeAdd = appendableSegment.getNumRowsInMemory();
+ final int appendableSegmentRowsInMemoryAfterAdd;
+ final long bytesInMemoryBeforeAdd = appendableSegment.getBytesInMemory();
final long bytesInMemoryAfterAdd;
final IncrementalIndexAddResult addResult;
try {
- addResult = sink.add(row, !allowIncrementalPersists);
- sinkRowsInMemoryAfterAdd = addResult.getRowCount();
+ addResult = appendableSegment.add(row, !allowIncrementalPersists);
+ appendableSegmentRowsInMemoryAfterAdd = addResult.getRowCount();
bytesInMemoryAfterAdd = addResult.getBytesInMemory();
}
catch (IndexSizeExceededException e) {
// Uh oh, we can't do anything about this! We can't persist (commit metadata would be out of sync) and we
// can't add the row (it just failed). This should never actually happen, though, because we check
- // sink.canAddRow after returning from add.
- log.error(e, "Sink for segment[%s] was unexpectedly full!", identifier);
+ // appendableSegment.canAddRow after returning from add.
+ log.error(e, "Appendable segment[%s] was unexpectedly full!", identifier);
throw e;
}
- if (sinkRowsInMemoryAfterAdd < 0) {
- throw new SegmentNotWritableException("Attempt to add row to swapped-out sink for segment[%s].", identifier);
+ if (appendableSegmentRowsInMemoryAfterAdd < 0) {
+ throw new SegmentNotWritableException("Attempt to add row to swapped-out appendable segment[%s].", identifier);
}
if (addResult.isRowAdded()) {
@@ -359,7 +361,7 @@ public AppenderatorAddResult add(
parseExceptionHandler.handle(addResult.getParseException());
}
- final int numAddedRows = sinkRowsInMemoryAfterAdd - sinkRowsInMemoryBeforeAdd;
+ final int numAddedRows = appendableSegmentRowsInMemoryAfterAdd - appendableSegmentRowsInMemoryBeforeAdd;
rowsCurrentlyInMemory.addAndGet(numAddedRows);
bytesCurrentlyInMemory.addAndGet(bytesInMemoryAfterAdd - bytesInMemoryBeforeAdd);
totalRows.addAndGet(numAddedRows);
@@ -368,9 +370,9 @@ public AppenderatorAddResult add(
boolean persist = false;
List persistReasons = new ArrayList<>();
- if (!sink.canAppendRow()) {
+ if (!appendableSegment.canAppendRow()) {
persist = true;
- persistReasons.add("No more rows can be appended to sink");
+ persistReasons.add("No more rows can be appended to appendable segment");
}
if (System.currentTimeMillis() > nextFlush) {
persist = true;
@@ -402,15 +404,16 @@ public AppenderatorAddResult add(
log.info("Flushing in-memory data to disk because %s.", String.join(",", persistReasons));
long bytesToBePersisted = 0L;
- for (Map.Entry entry : sinks.entrySet()) {
- final Sink sinkEntry = entry.getValue();
- if (sinkEntry != null) {
- bytesToBePersisted += sinkEntry.getBytesInMemory();
- if (sinkEntry.swappable()) {
- // After swapping the sink, we use memory mapped segment instead (but only for real time appenderators!).
- // However, the memory mapped segment still consumes memory.
- // These memory mapped segments are held in memory throughout the ingestion phase and permanently add to the bytesCurrentlyInMemory
- int memoryStillInUse = calculateMMappedHydrantMemoryInUsed(sink.getCurrHydrant());
+ for (Map.Entry entry : appendableSegments.entrySet()) {
+ final AppendableSegment appendableSegmentEntry = entry.getValue();
+ if (appendableSegmentEntry != null) {
+ bytesToBePersisted += appendableSegmentEntry.getBytesInMemory();
+ if (appendableSegmentEntry.swappable()) {
+ // After swapping the appendable segment, we use memory mapped segment instead (but only for real time
+ // appenderators!). However, the memory mapped segment still consumes memory. These memory mapped
+ // segments are held in memory throughout the ingestion phase and permanently add to the
+ // bytesCurrentlyInMemory
+ int memoryStillInUse = calculateMMappedPartialSegmentsMemoryInUsed(appendableSegment.getCurrentPartialSegment());
bytesCurrentlyInMemory.addAndGet(memoryStillInUse);
}
}
@@ -421,10 +424,10 @@ public AppenderatorAddResult add(
// This means that we ran out of all available memory to ingest (due to overheads created as part of ingestion)
final String alertMessage = StringUtils.format(
"Task has exceeded safe estimated heap usage limits, failing "
- + "(numSinks: [%d] numHydrantsAcrossAllSinks: [%d] totalRows: [%d])"
+ + "(numAppendableSegments: [%d] numPartialSegmentsAcrossAllAppendableSegments: [%d] totalRows: [%d])"
+ "(bytesCurrentlyInMemory: [%d] - bytesToBePersisted: [%d] > maxBytesTuningConfig: [%d])",
- sinks.size(),
- sinks.values().stream().mapToInt(Iterables::size).sum(),
+ appendableSegments.size(),
+ appendableSegments.values().stream().mapToInt(Iterables::size).sum(),
getTotalRowCount(),
bytesCurrentlyInMemory.get(),
bytesToBePersisted,
@@ -469,24 +472,24 @@ public void onFailure(Throwable t)
isPersistRequired = true;
}
}
- return new AppenderatorAddResult(identifier, sink.getNumRows(), isPersistRequired);
+ return new AppenderatorAddResult(identifier, appendableSegment.getNumRows(), isPersistRequired);
}
@Override
public List getSegments()
{
- return ImmutableList.copyOf(sinks.keySet());
+ return ImmutableList.copyOf(appendableSegments.keySet());
}
@Override
public int getRowCount(final SegmentIdWithShardSpec identifier)
{
- final Sink sink = sinks.get(identifier);
+ final AppendableSegment appendableSegment = appendableSegments.get(identifier);
- if (sink == null) {
- throw new ISE("No such sink: %s", identifier);
+ if (appendableSegment == null) {
+ throw new ISE("No such appendable segment: %s", identifier);
} else {
- return sink.getNumRows();
+ return appendableSegment.getNumRows();
}
}
@@ -511,21 +514,21 @@ long getBytesCurrentlyInMemory()
@VisibleForTesting
long getBytesInMemory(SegmentIdWithShardSpec identifier)
{
- final Sink sink = sinks.get(identifier);
+ final AppendableSegment appendableSegment = appendableSegments.get(identifier);
- if (sink == null) {
- throw new ISE("No such sink: %s", identifier);
+ if (appendableSegment == null) {
+ throw new ISE("No such appendable segment: %s", identifier);
} else {
- return sink.getBytesInMemory();
+ return appendableSegment.getBytesInMemory();
}
}
- private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier)
+ private AppendableSegment getOrCreateAppendableSegment(final SegmentIdWithShardSpec identifier)
{
- Sink retVal = sinks.get(identifier);
+ AppendableSegment retVal = appendableSegments.get(identifier);
if (retVal == null) {
- retVal = new Sink(
+ retVal = new AppendableSegment(
identifier.getInterval(),
schema,
identifier.getShardSpec(),
@@ -535,7 +538,7 @@ private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier)
maxBytesTuningConfig,
useMaxMemoryEstimates
);
- bytesCurrentlyInMemory.addAndGet(calculateSinkMemoryInUsed(retVal));
+ bytesCurrentlyInMemory.addAndGet(calculateAppendableSegmentMemoryInUsed(retVal));
try {
segmentAnnouncer.announceSegment(retVal.getSegment());
@@ -546,7 +549,7 @@ private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier)
.emit();
}
- addSink(identifier, retVal);
+ addAppendableSegment(identifier, retVal);
}
return retVal;
@@ -599,7 +602,7 @@ public void clear() throws InterruptedException
// Drop everything.
final List> futures = new ArrayList<>();
- for (Map.Entry entry : sinks.entrySet()) {
+ for (Map.Entry