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..dc2ddea1ba5e 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 segments not yet 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..60d8f686a28c 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 @@ -33,9 +33,9 @@ import org.apache.druid.initialization.Initialization; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; 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.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 95% 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..f35e9e2fd44a 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 @@ -17,7 +17,7 @@ * 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 +47,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 +64,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 +106,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 +134,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 +167,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"), 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 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..9732d72eaf5e 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 @@ -58,9 +58,9 @@ import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandlerProvider; 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.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..7190c38849f2 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 @@ -59,8 +59,8 @@ import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.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..b71a98c63e74 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; @@ -87,6 +84,7 @@ import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.realtime.ChatHandler; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; @@ -95,7 +93,6 @@ 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.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..c29708fb5009 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 @@ -75,10 +75,10 @@ import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandler; +import org.apache.druid.segment.realtime.ChatHandlers; 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.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..8d49a7f3dbe7 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 @@ -60,13 +60,13 @@ import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandler; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; 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.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..37a95ea14de7 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 @@ -49,10 +49,10 @@ import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.realtime.ChatHandler; 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.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..81ae8e5c4fde 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 @@ -81,13 +81,13 @@ import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.realtime.ChatHandler; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; 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.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..a4f297c8e335 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 @@ -51,9 +51,9 @@ import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLocalCacheManager; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; 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.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 seedRows) - { - this.parser = parser; - this.closed = !waitForClose; - - if (parser instanceof StringInputRowParser) { - ((StringInputRowParser) parser).startFileFromBeginning(); - } - - if (seedRows != null) { - seedRows.stream().map(Optional::ofNullable).forEach(queue::add); - } - } - - public void addRows(List rows) - { - synchronized (this) { - rows.stream().map(Optional::ofNullable).forEach(queue::add); - notifyAll(); - } - } - - @Override - public boolean hasMore() - { - try { - synchronized (this) { - while (queue.isEmpty() && !closed) { - wait(); - } - return !queue.isEmpty(); - } - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } - - @Override - public InputRow nextRow() - { - synchronized (this) { - final InputRow row = parser instanceof StringInputRowParser - ? ((StringInputRowParser) parser).parse((String) queue.removeFirst().orElse(null)) - : (InputRow) parser.parseBatch(queue.removeFirst().orElse(null)).get(0); - if (row != null && row.getRaw(FAIL_DIM) != null) { - throw new ParseException(null, FAIL_DIM); - } - return row; - } - } - - @Override - public void close() - { - synchronized (this) { - closed = true; - notifyAll(); - } - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java index b4166b7bc292..2643e60147de 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java @@ -63,7 +63,6 @@ public TestIndexTask( new IndexIngestionSpec( new DataSchema(dataSource, null, new AggregatorFactory[]{}, null, null, mapper), new IndexTask.IndexIOConfig( - null, new LocalInputSource(new File("lol"), "rofl"), new JsonInputFormat(null, null, null, null, null), false, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java index 7dda3b8ff62c..ae5ba44f7f09 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java @@ -20,7 +20,6 @@ package org.apache.druid.indexing.common; import com.fasterxml.jackson.databind.InjectableValues; -import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; @@ -30,7 +29,6 @@ import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.NoopInputSource; import org.apache.druid.guice.DruidSecondaryModule; -import org.apache.druid.guice.FirehoseModule; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClientProvider; @@ -49,15 +47,14 @@ import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalLoadSpec; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; 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.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; -import java.util.List; import java.util.concurrent.TimeUnit; /** @@ -122,9 +119,6 @@ public void setupModule(SetupContext context) } ); DruidSecondaryModule.setupAnnotationIntrospector(jsonMapper, TestHelper.makeAnnotationIntrospector()); - - List firehoseModules = new FirehoseModule().getJacksonModules(); - firehoseModules.forEach(jsonMapper::registerModule); } public ObjectMapper getTestObjectMapper() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java index f789c1e641f7..278a27f24bba 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java @@ -52,12 +52,12 @@ import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; import org.apache.druid.segment.realtime.appenderator.BatchAppenderator; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.server.security.AuthTestUtils; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index 71514af17b42..33278b23db70 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -60,9 +60,9 @@ import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; 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.writeout.TmpFileSegmentWriteOutMediumFactory; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.server.security.AuthTestUtils; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index ba9a6e3e2be2..694b8cf033c0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -939,7 +939,6 @@ protected TaskToolbox createTaskToolbox(Task task, TaskActionClient actionClient private void runIndexTask(@Nullable PartitionsSpec partitionsSpec, boolean appendToExisting) { ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(inputDir, "druid*"), new CsvInputFormat( Arrays.asList("ts", "dim", "val"), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 599a24fac802..d089c2cc24a2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -97,8 +97,8 @@ import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.loading.TombstoneLoadSpec; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; +import org.apache.druid.segment.realtime.WindowedStorageAdapter; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 134f5305169d..ec2f73a0b8ee 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -127,9 +127,9 @@ import org.apache.druid.segment.loading.NoopSegmentCacheManager; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; 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.selector.settable.SettableColumnValueSelector; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java index 7555c6b1e378..ab953ba954ac 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java @@ -54,7 +54,6 @@ public void testParserAndInputFormat() null ), new IndexIOConfig( - null, new NoopInputSource(), new NoopInputFormat(), null, @@ -79,7 +78,6 @@ public void testParserAndInputSource() null ), new IndexIOConfig( - null, new NoopInputSource(), null, null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index f0e0198e35b0..316d6a40f4a5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -93,7 +93,7 @@ import org.apache.druid.segment.loading.SegmentLocalCacheManager; import org.apache.druid.segment.loading.StorageLocation; import org.apache.druid.segment.loading.StorageLocationConfig; -import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.WindowedStorageAdapter; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -244,7 +244,6 @@ public void testCorrectInputSourceResources() null ), new IndexIOConfig( - null, new LocalInputSource(tmpDir, "druid*"), DEFAULT_INPUT_FORMAT, false, @@ -295,7 +294,6 @@ public void testIngestNullOnlyColumns() throws Exception null ), new IndexIOConfig( - null, new LocalInputSource(tmpDir, "druid*"), DEFAULT_INPUT_FORMAT, false, @@ -358,7 +356,6 @@ public void testIngestNullOnlyColumns_storeEmptyColumnsOff_shouldNotStoreEmptyCo null ), new IndexIOConfig( - null, new LocalInputSource(tmpDir, "druid*"), DEFAULT_INPUT_FORMAT, false, @@ -2794,7 +2791,6 @@ private static IndexIngestionSpec createIngestionSpec( transformSpec ), new IndexIOConfig( - null, new LocalInputSource(baseDir, "druid*"), inputFormat, appendToExisting, @@ -2822,7 +2818,6 @@ private static IndexIngestionSpec createIngestionSpec( objectMapper ), new IndexIOConfig( - null, new LocalInputSource(baseDir, "druid*"), createInputFormatFromParseSpec(parseSpec), appendToExisting, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 133ced3907dc..fa2a065ac7fc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -23,7 +23,6 @@ import com.google.common.base.Optional; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.CsvInputFormat; @@ -86,7 +85,7 @@ import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; -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.metrics.NoopServiceEmitter; import org.apache.druid.server.security.AuthTestUtils; @@ -294,8 +293,8 @@ public IndexMergerV9Factory getIndexMergerV9Factory() } /** - * Converts ParseSpec to InputFormat for indexing tests. To be used until {@link FirehoseFactory} - * & {@link InputRowParser} is deprecated and removed. + * Converts ParseSpec to InputFormat for indexing tests. To be used until the {@link InputRowParser} is removed + * This is already deprecated, but the translation logic still remains. */ public static InputFormat createInputFormatFromParseSpec(ParseSpec parseSpec) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java index b217f8d5bd83..8023eb646d8f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java @@ -19,19 +19,14 @@ package org.apache.druid.indexing.common.task; -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; import com.google.common.collect.ImmutableMap; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.FirehoseModule; import org.apache.druid.indexer.HadoopIOConfig; import org.apache.druid.indexer.HadoopIngestionSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; @@ -71,13 +66,9 @@ public TaskSerdeTest() TestUtils testUtils = new TestUtils(); jsonMapper = testUtils.getTestObjectMapper(); - for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) { - jsonMapper.registerModule(jacksonModule); - } jsonMapper.registerSubtypes( new NamedType(ParallelIndexTuningConfig.class, "index_parallel"), - new NamedType(IndexTuningConfig.class, "index"), - new NamedType(MockFirehoseFactory.class, "mock") + new NamedType(IndexTuningConfig.class, "index") ); } @@ -242,7 +233,7 @@ public void testIndexTaskSerde() throws Exception ), null ), - new IndexIOConfig(null, new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false), + new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false), new IndexTuningConfig( null, null, @@ -329,7 +320,7 @@ public void testIndexTaskwithResourceSerde() throws Exception ), null ), - new IndexIOConfig(null, new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false), + new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false), new IndexTuningConfig( null, null, @@ -361,10 +352,6 @@ public void testIndexTaskwithResourceSerde() throws Exception null ); - for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) { - jsonMapper.registerModule(jacksonModule); - } - final String json = jsonMapper.writeValueAsString(task); Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change @@ -501,15 +488,6 @@ public void testHadoopIndexTaskSerde() throws Exception Assert.assertEquals("blah", task2.getClasspathPrefix()); } - private static class MockFirehoseFactory implements FirehoseFactory - { - @Override - public Firehose connect(InputRowParser parser, File temporaryDirectory) - { - return null; - } - } - @Test public void testHadoopIndexTaskWithContextSerde() throws Exception { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java index be322ad06d38..c03abc3c917c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java @@ -218,7 +218,6 @@ protected ParallelIndexSupervisorTask createTask( if (useInputFormatApi) { Preconditions.checkArgument(parseSpec == null); ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(inputDirectory, filter), inputFormat, appendToExisting, @@ -239,7 +238,6 @@ protected ParallelIndexSupervisorTask createTask( } else { Preconditions.checkArgument(inputFormat == null && parseSpec != null); ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(inputDirectory, filter), createInputFormatFromParseSpec(parseSpec), appendToExisting, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index ecc4f702d6ae..9de10d130274 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -96,10 +96,10 @@ import org.apache.druid.segment.loading.NoopDataSegmentKiller; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthTestUtils; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java index e71dc6db36d3..b8c59d042a31 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java @@ -244,7 +244,6 @@ private ParallelIndexSupervisorTask createTestTask( if (useInputFormatApi) { Preconditions.checkArgument(parseSpec == null); ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(inputDir, filter), inputFormat, appendToExisting, @@ -265,7 +264,6 @@ private ParallelIndexSupervisorTask createTestTask( } else { Preconditions.checkArgument(inputFormat == null); ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(inputDir, filter), createInputFormatFromParseSpec(parseSpec), appendToExisting, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java index 1448a5723f36..0d19cd86e03c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java @@ -140,7 +140,6 @@ public void testIngestNullColumn() throws JsonProcessingException null ), new ParallelIndexIOConfig( - null, getInputSource(), JSON_FORMAT, false, @@ -191,7 +190,6 @@ public void testIngestNullColumn_useFieldDiscovery_includeAllDimensions_shouldSt null ), new ParallelIndexIOConfig( - null, getInputSource(), new JsonInputFormat( new JSONPathSpec(true, null), @@ -252,7 +250,6 @@ public void testIngestNullColumn_explicitPathSpec_useFieldDiscovery_includeAllDi null ), new ParallelIndexIOConfig( - null, getInputSource(), new JsonInputFormat( new JSONPathSpec( @@ -321,7 +318,6 @@ public void testIngestNullColumn_storeEmptyColumnsOff_shouldNotStoreEmptyColumns null ), new ParallelIndexIOConfig( - null, getInputSource(), JSON_FORMAT, false, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java index a32aed819e0c..5ab85f38b072 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java @@ -80,7 +80,6 @@ public void testStopGracefully() throws Exception final ParallelIndexSupervisorTask task = newTask( Intervals.of("2017/2018"), new ParallelIndexIOConfig( - null, // Sub tasks would run forever new TestInputSource(Pair.of(new TestInput(Integer.MAX_VALUE, TaskState.SUCCESS), 4)), new NoopInputFormat(), @@ -111,7 +110,6 @@ public void testSubTaskFail() throws Exception final ParallelIndexSupervisorTask task = newTask( Intervals.of("2017/2018"), new ParallelIndexIOConfig( - null, new TestInputSource( Pair.of(new TestInput(10L, TaskState.FAILED), 1), Pair.of(new TestInput(Integer.MAX_VALUE, TaskState.FAILED), 3) @@ -331,7 +329,6 @@ SinglePhaseSubTaskSpec newTaskSpec(InputSplit split) new ParallelIndexIngestionSpec( getIngestionSchema().getDataSchema(), new ParallelIndexIOConfig( - null, baseInputSource.withSplit(split), getIngestionSchema().getIOConfig().getInputFormat(), getIngestionSchema().getIOConfig().isAppendToExisting(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index 772bdafb2b17..ce6dfd9e89d9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -130,7 +130,6 @@ public void testAPIs() throws Exception task = newTask( Intervals.of("2017/2018"), new ParallelIndexIOConfig( - null, new TestInputSource(IntStream.range(0, NUM_SUB_TASKS).boxed().collect(Collectors.toList())), new NoopInputFormat(), false, @@ -563,7 +562,6 @@ SinglePhaseSubTaskSpec newTaskSpec(InputSplit split) new ParallelIndexIngestionSpec( getIngestionSchema().getDataSchema(), new ParallelIndexIOConfig( - null, baseInputSource.withSplit(split), getIngestionSchema().getIOConfig().getInputFormat(), getIngestionSchema().getIOConfig().isAppendToExisting(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java index 7b92ba44bd9d..f5208f4bf5a4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java @@ -190,7 +190,6 @@ private static class ParallelIndexIngestionSpecBuilder ); private final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(new File("tmp"), "test_*"), new CsvInputFormat(Arrays.asList("ts", "dim", "val"), null, null, false, 0), false, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java index d763337978fb..26d672b0eb67 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java @@ -230,7 +230,6 @@ public void testFailToConstructWhenBothAppendToExistingAndForceGuaranteedRollupA final boolean appendToExisting = true; final boolean forceGuaranteedRollup = true; final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new InlineInputSource("test"), new JsonInputFormat(null, null, null, null, null), appendToExisting, @@ -303,7 +302,6 @@ public void testFailToConstructWhenBothInputSourceAndParserAreSet() { final ObjectMapper mapper = new DefaultObjectMapper(); final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new InlineInputSource("test"), null, false, @@ -553,8 +551,7 @@ public void testCompactionTaskDoesntCleanup() throws Exception final boolean appendToExisting = false; final boolean forceGuaranteedRollup = true; final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, - new InlineInputSource("test"), + new InlineInputSource("test"), new JsonInputFormat(null, null, null, null, null), appendToExisting, null diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java index 0a0d4d4692f0..09877c680c31 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java @@ -217,7 +217,7 @@ static ParallelIndexIngestionSpec createIngestionSpec( DataSchema dataSchema ) { - ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(null, inputSource, inputFormat, false, false); + ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(inputSource, inputFormat, false, false); return new ParallelIndexIngestionSpec(dataSchema, ioConfig, tuningConfig); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java index 97e4f54c06ef..96494b8ac794 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java @@ -323,7 +323,6 @@ protected ParallelIndexSupervisorTask newTask( Preconditions.checkArgument(parseSpec == null); ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new LocalInputSource(inputDir, filter), inputFormat, appendToExisting, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index 5ad774386b65..8f25b7ddbb75 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -405,7 +405,6 @@ public void testRunInParallelIngestNullColumn() null ), new ParallelIndexIOConfig( - null, new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true), DEFAULT_INPUT_FORMAT, false, @@ -459,7 +458,6 @@ public void testRunInParallelIngestNullColumn_storeEmptyColumnsOff_shouldNotStor null ), new ParallelIndexIOConfig( - null, new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true), DEFAULT_INPUT_FORMAT, false, @@ -893,7 +891,6 @@ public void testIngestBothExplicitAndImplicitDims() throws IOException null ), new ParallelIndexIOConfig( - null, new SettableSplittableLocalInputSource(inputDir, "*.json", true), new JsonInputFormat( new JSONPathSpec(true, null), @@ -977,7 +974,6 @@ public void testIngestBothExplicitAndImplicitDimsSchemaDiscovery() throws IOExce null ), new ParallelIndexIOConfig( - null, new SettableSplittableLocalInputSource(inputDir, "*.json", true), new JsonInputFormat( new JSONPathSpec(true, null), @@ -1058,7 +1054,6 @@ private ParallelIndexSupervisorTask newTask( null ), new ParallelIndexIOConfig( - null, new SettableSplittableLocalInputSource(inputDir, inputSourceFilter, splittableInputSource), DEFAULT_INPUT_FORMAT, appendToExisting, @@ -1081,7 +1076,6 @@ private ParallelIndexSupervisorTask newTask( null ), new ParallelIndexIOConfig( - null, new LocalInputSource(inputDir, inputSourceFilter), createInputFormatFromParseSpec(DEFAULT_PARSE_SPEC), appendToExisting, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpecTest.java index b70cc7a99e69..4acc3d3f5912 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpecTest.java @@ -57,7 +57,6 @@ public class SinglePhaseSubTaskSpecTest null ), new ParallelIndexIOConfig( - null, new LocalInputSource(new File("baseDir"), "filter"), new JsonInputFormat(null, null, null, null, null), null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java index ad34aa10ebe9..5ee6ac6f4c17 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java @@ -52,7 +52,7 @@ import org.apache.druid.segment.loading.NoopDataSegmentMover; import org.apache.druid.segment.loading.NoopDataSegmentPusher; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -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.SetAndVerifyContextQueryRunner; import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 91d74a2bd677..c3aa999e9a18 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -37,8 +37,6 @@ import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.data.input.AbstractInputSource; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; @@ -46,7 +44,6 @@ import org.apache.druid.data.input.InputStats; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.discovery.DataNodeService; @@ -131,9 +128,9 @@ import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.NoopDataSegmentArchiver; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; 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.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.coordination.ServerType; @@ -145,9 +142,7 @@ import org.apache.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMock; import org.joda.time.DateTime; -import org.joda.time.Hours; import org.joda.time.Interval; -import org.joda.time.Period; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -216,12 +211,6 @@ public int compare(DataSegment dataSegment, DataSegment dataSegment2) }; private static DateTime now = DateTimes.nowUtc(); - private static final Iterable REALTIME_IDX_TASK_INPUT_ROWS = ImmutableList.of( - ir(now.toString("YYYY-MM-dd'T'HH:mm:ss"), "test_dim1", "test_dim2", 1.0f), - ir(now.plus(new Period(Hours.ONE)).toString("YYYY-MM-dd'T'HH:mm:ss"), "test_dim1", "test_dim2", 2.0f), - ir(now.plus(new Period(Hours.TWO)).toString("YYYY-MM-dd'T'HH:mm:ss"), "test_dim1", "test_dim2", 3.0f) - ); - private static final Iterable IDX_TASK_INPUT_ROWS = ImmutableList.of( ir("2010-01-01T01", "x", "y", 1), ir("2010-01-01T01", "x", "z", 1), @@ -258,7 +247,7 @@ public int compare(DataSegment dataSegment, DataSegment dataSegment2) druidNode.getTlsPort() ); private int pushedSegments; - private int announcedSinks; + private int announcedAppendableSegments; private SegmentHandoffNotifierFactory handoffNotifierFactory; private Map> handOffCallbacks; @@ -369,37 +358,6 @@ public boolean needsFormat() } } - private static class MockFirehoseFactory implements FirehoseFactory - { - @Override - public Firehose connect(InputRowParser parser, File temporaryDirectory) - { - final Iterator inputRowIterator = REALTIME_IDX_TASK_INPUT_ROWS.iterator(); - - return new Firehose() - { - @Override - public boolean hasMore() - { - return inputRowIterator.hasNext(); - } - - @Nullable - @Override - public InputRow nextRow() - { - return inputRowIterator.next(); - } - - @Override - public void close() - { - - } - }; - } - } - @Before public void setUp() throws Exception { @@ -408,7 +366,7 @@ public void setUp() throws Exception monitorScheduler = EasyMock.createStrictMock(MonitorScheduler.class); // initialize variables - announcedSinks = 0; + announcedAppendableSegments = 0; pushedSegments = 0; indexSpec = IndexSpec.DEFAULT; emitter = newMockEmitter(); @@ -453,7 +411,6 @@ private TaskStorage setUpTaskStorage() case METADATA_TASK_STORAGE: { TestDerbyConnector testDerbyConnector = derbyConnectorRule.getConnector(); mapper.registerSubtypes( - new NamedType(MockFirehoseFactory.class, "mockFirehoseFactory"), new NamedType(MockInputSource.class, "mockInputSource"), new NamedType(NoopInputFormat.class, "noopInputFormat") ); @@ -625,7 +582,7 @@ private TaskToolboxFactory setUpTaskToolboxFactory( @Override public void announceSegment(DataSegment segment) { - announcedSinks++; + announcedAppendableSegments++; } }, @@ -729,7 +686,7 @@ public void testIndexTask() ), null ), - new IndexIOConfig(null, new MockInputSource(), new NoopInputFormat(), false, false), + new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), new IndexTuningConfig( null, 10000, @@ -814,7 +771,7 @@ public void testIndexTaskFailure() null, mapper ), - new IndexIOConfig(null, new MockExceptionInputSource(), new NoopInputFormat(), false, false), + new IndexIOConfig(new MockExceptionInputSource(), new NoopInputFormat(), false, false), new IndexTuningConfig( null, 10000, @@ -1258,7 +1215,7 @@ public void testResumeTasks() throws Exception ), null ), - new IndexIOConfig(null, new MockInputSource(), new NoopInputFormat(), false, false), + new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), new IndexTuningConfig( null, 10000, @@ -1370,7 +1327,7 @@ public void testUnifiedAppenderatorsManagerCleanup() throws Exception ), null ), - new IndexIOConfig(null, new MockInputSource(), new NoopInputFormat(), false, false), + new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), new IndexTuningConfig( null, 10000, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java index 8984e73b6b9c..ee90a3335a17 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java @@ -558,7 +558,6 @@ public void testGetActiveTaskRedactsPassword() throws JsonProcessingException null ); final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, new HttpInputSource(Collections.singletonList(URI.create("http://host.org")), "user", new DefaultPasswordProvider(password), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java index edebc0fe8dc8..19e4bd33fd8e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java @@ -57,8 +57,8 @@ import org.apache.druid.segment.loading.DataSegmentMover; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentAnnouncer; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java index 6f92aeef646b..8fe2f5fd044a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java @@ -28,7 +28,6 @@ import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; -import org.apache.druid.guice.FirehoseModule; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.indexing.DataSchema; @@ -65,7 +64,6 @@ public IndexTaskSamplerSpecTest() .addValue(ObjectMapper.class, MAPPER) ); MAPPER.registerModules((Iterable) new SamplerModule().getJacksonModules()); - MAPPER.registerModules((Iterable) new FirehoseModule().getJacksonModules()); } @Test diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index 1a9e5a17e1cf..6421aa697778 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -118,8 +118,8 @@ import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.SegmentSchemaManager; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.StreamAppenderator; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.coordination.ServerType; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java index ca3cd6edf8b3..88249509ef97 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java @@ -46,7 +46,7 @@ public TaskAnnouncementTest() @Test public void testBackwardsCompatibleSerde() throws Exception { - final IndexTask.IndexIOConfig ioConfig = new IndexTask.IndexIOConfig(null, new NoopInputSource(), null, null, null); + final IndexTask.IndexIOConfig ioConfig = new IndexTask.IndexIOConfig(new NoopInputSource(), null, null, null); final Task task = new IndexTask( "theid", new TaskResource("rofl", 2), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java index 8217a12eb837..3eea9d33fb3e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java @@ -52,7 +52,7 @@ import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.server.coordination.ChangeRequestHistory; import org.apache.druid.server.coordination.ChangeRequestsSnapshot; import org.apache.druid.server.security.AuthTestUtils; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java index c3191cc068d5..fcdada15f5da 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -55,7 +55,7 @@ import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -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.initialization.IndexerZkConfig; import org.apache.druid.server.initialization.ServerConfig; diff --git a/integration-tests-ex/docs/conversion.md b/integration-tests-ex/docs/conversion.md index 6a14533d3c4c..0fe4a7348e42 100644 --- a/integration-tests-ex/docs/conversion.md +++ b/integration-tests-ex/docs/conversion.md @@ -112,7 +112,7 @@ The `docker-tests/pom.xml` file includes Maven dependencies for the most common Druid modules, which transitiviely include the third-party modules which the Druid modules reference. You test sub-project may need addition dependencies. To find them, review `integration-tests/pom.xml`. Careful, -however, as that file is a bit of a "kitchen sink" that includes every +however, as that file is a bit of a "kitchen appendableSegment" that includes every possible dependency, even those already available transitively. If you feel the dependency is one used by multiple tests, go ahead and diff --git a/integration-tests-ex/docs/dependencies.md b/integration-tests-ex/docs/dependencies.md index 90047449f5cb..1fb92392e881 100644 --- a/integration-tests-ex/docs/dependencies.md +++ b/integration-tests-ex/docs/dependencies.md @@ -93,7 +93,7 @@ case: the server. The result is either a huge amount of time fiddling to get things right or (as in the original integration tests), we just include everything and pretend we are a server. -There is no obvious solution, it is just a massive time sink at +There is no obvious solution, it is just a massive time appendableSegment at present. ### Druid Modules diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/EventReceiverFirehoseTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/EventReceiverFirehoseTestClient.java deleted file mode 100644 index 761d9f55dbaf..000000000000 --- a/integration-tests/src/main/java/org/apache/druid/testing/clients/EventReceiverFirehoseTestClient.java +++ /dev/null @@ -1,181 +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.testing.clients; - -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.jackson.JacksonUtils; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.java.util.http.client.Request; -import org.apache.druid.java.util.http.client.response.StatusResponseHandler; -import org.apache.druid.java.util.http.client.response.StatusResponseHolder; -import org.apache.druid.testing.guice.TestClient; -import org.jboss.netty.handler.codec.http.HttpMethod; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; - -import javax.ws.rs.core.MediaType; -import java.io.BufferedReader; -import java.io.InputStreamReader; -import java.net.URL; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Map; -import java.util.concurrent.ExecutionException; - -public class EventReceiverFirehoseTestClient -{ - private static final Logger LOG = new Logger(EventReceiverFirehoseTestClient.class); - - static final int NUM_RETRIES = 30; - static final long DELAY_FOR_RETRIES_MS = 10000; - - private final String host; - private final ObjectMapper jsonMapper; - private final HttpClient httpClient; - private final String chatID; - private final ObjectMapper smileMapper; - - public EventReceiverFirehoseTestClient( - String host, - String chatID, - ObjectMapper jsonMapper, - @TestClient HttpClient httpClient, - ObjectMapper smileMapper - ) - { - this.host = host; - this.jsonMapper = jsonMapper; - this.httpClient = httpClient; - this.chatID = chatID; - this.smileMapper = smileMapper; - } - - private String getURL() - { - return StringUtils.format( - "https://%s/druid/worker/v1/chat/%s/push-events/", - host, - chatID - ); - } - - /** - * post events from the collection and return the count of events accepted - * - * @param events Collection of events to be posted - * - * @return - */ - public int postEvents(Collection> events, ObjectMapper objectMapper, String mediaType) - throws InterruptedException - { - int retryCount = 0; - while (true) { - try { - StatusResponseHolder response = httpClient.go( - new Request(HttpMethod.POST, new URL(getURL())) - .setContent(mediaType, objectMapper.writeValueAsBytes(events)), - StatusResponseHandler.getInstance() - ).get(); - - if (!response.getStatus().equals(HttpResponseStatus.OK)) { - throw new ISE( - "Error while posting events to url[%s] status[%s] content[%s]", - getURL(), - response.getStatus(), - response.getContent() - ); - } - Map responseData = objectMapper.readValue( - response.getContent(), new TypeReference>() - { - } - ); - return responseData.get("eventCount"); - } - // adding retries to flaky tests using channels - catch (ExecutionException e) { - if (retryCount > NUM_RETRIES) { - throw new RuntimeException(e); //giving up now - } else { - LOG.info(e, "received exception, sleeping and retrying"); - retryCount++; - Thread.sleep(DELAY_FOR_RETRIES_MS); - } - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - } - - /** - * Reads each events from file and post them to the indexing service. - * Uses both smileMapper and jsonMapper to send events alternately. - * - * @param file location of file to post events from - * - * @return number of events sent to the indexing service - */ - public int postEventsFromFile(String file) - { - try ( - BufferedReader reader = new BufferedReader( - new InputStreamReader( - EventReceiverFirehoseTestClient.class.getResourceAsStream(file), - StandardCharsets.UTF_8 - ) - ) - ) { - - String s; - Collection> events = new ArrayList>(); - // Test sending events using both jsonMapper and smileMapper. - // sends events one by one using both jsonMapper and smileMapper. - int totalEventsPosted = 0; - int expectedEventsPosted = 0; - while ((s = reader.readLine()) != null) { - events.add(this.jsonMapper.readValue(s, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT)); - ObjectMapper mapper = (totalEventsPosted % 2 == 0) ? jsonMapper : smileMapper; - String mediaType = (totalEventsPosted % 2 == 0) - ? MediaType.APPLICATION_JSON - : SmileMediaTypes.APPLICATION_JACKSON_SMILE; - totalEventsPosted += postEvents(events, mapper, mediaType); - - expectedEventsPosted += events.size(); - events = new ArrayList<>(); - } - - if (totalEventsPosted != expectedEventsPosted) { - throw new ISE("All events not posted, expected : %d actual : %d", events.size(), totalEventsPosted); - } - return totalEventsPosted; - } - catch (Exception e) { - throw new RuntimeException(e); - } - - } -} diff --git a/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModuleFactory.java b/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModuleFactory.java index d08712dea35e..1195795cd970 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModuleFactory.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/guice/DruidTestModuleFactory.java @@ -23,7 +23,6 @@ import com.google.inject.Injector; import com.google.inject.Module; import org.apache.druid.guice.GuiceInjectors; -import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.IndexingServiceTuningConfigModule; import org.apache.druid.initialization.Initialization; @@ -50,7 +49,6 @@ private static List getModules() { return ImmutableList.of( new DruidTestModule(), - new IndexingServiceFirehoseModule(), new IndexingServiceInputSourceModule(), new IndexingServiceTuningConfigModule() ); diff --git a/processing/src/main/java/org/apache/druid/data/input/Firehose.java b/processing/src/main/java/org/apache/druid/data/input/Firehose.java deleted file mode 100644 index a9ff6f33159c..000000000000 --- a/processing/src/main/java/org/apache/druid/data/input/Firehose.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.data.input; - -import org.apache.druid.guice.annotations.ExtensionPoint; -import org.apache.druid.java.util.common.parsers.ParseException; - -import javax.annotation.Nullable; -import java.io.Closeable; -import java.io.IOException; - -/** - * This is an interface that holds onto the stream of incoming data. Realtime data ingestion is built around this - * abstraction. - * - * This object acts a lot like an Iterator, but it doesn't extend the Iterator interface because it extends - * Closeable and it is very important that the {@link #close()} method doesn't get forgotten, which is easy to do if - * this gets passed around as an Iterator. Note that {@link #close()} doesn't cut the stream of rows for Firehose users - * immediately, but rather stops the supply of new rows into internal buffers. {@link #hasMore()} and {@link #nextRow()} - * are expected to operate for some time after (or concurrently with) {@link #close()} until the buffered events (if - * any) run out. - * - * Concurrency: - * The two methods {@link #hasMore()} and {@link #nextRow()} are all called from the same thread. - * {@link #close()} might be called concurrently from a thread different from the thread calling {@link #hasMore()} - * and {@link #nextRow()}. - *

- */ -@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..35166f5f762d 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 @@ -32,6 +32,7 @@ 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.query.AppendableSegmentQueryRunners; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryProcessingPool; @@ -40,7 +41,6 @@ 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.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/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index 1c9a6c67ded8..236307cc276c 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -39,6 +39,7 @@ import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.query.spec.SpecificSegmentSpec; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Rule; @@ -83,7 +84,7 @@ public int getMaxSegmentPartitionsOrderedInMemory() ); @RunWith(Parameterized.class) - public static class ScanQueryRunnerFactoryParameterizedTest + public static class ScanQueryRunnerFactoryParameterizedTest extends InitializedNullHandlingTest { private int numElements; private ScanQuery query; 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 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..795663e83ae9 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; } @@ -507,32 +507,33 @@ public short getAtomicUpdateGroupSize() * Helper for {@link #acquireSegmentReferences(Function, boolean)}. Separate method to simplify testing (we test this * method instead of testing {@link #acquireSegmentReferences(Function, boolean)} directly). */ + @Nullable @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 +545,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 85% 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..906d98f89f5a 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,16 @@ 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; + @Nullable 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 +57,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 +151,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 +170,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 +192,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 +221,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..0d673cf9c1f8 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 @@ -33,6 +33,7 @@ import org.apache.druid.java.util.common.guava.FunctionalIterable; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.query.AppendableSegmentQueryRunners; import org.apache.druid.query.BySegmentQueryRunner; import org.apache.druid.query.CPUTimeMetricQueryRunner; import org.apache.druid.query.DataSource; @@ -52,15 +53,14 @@ 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.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..c2c7258c6c98 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() + @JsonProperty("hydrants") + 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 entry : appendableSegments.entrySet()) { futures.add(abandonSegment(entry.getKey(), entry.getValue(), true)); } @@ -615,9 +618,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); } @@ -627,40 +630,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()); @@ -676,16 +679,16 @@ 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 segment[%s].", commitMetadata, Joiner.on(", ").join( - currentHydrants.entrySet() + currentPartialSegments.entrySet() .stream() .map(entry -> StringUtils.format( "%s:%d", @@ -700,14 +703,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 partial segments with existing partial segments + commitPartialSegments.putAll(oldCommit.getPartialSegments()); } - commitHydrants.putAll(currentHydrants); - writeCommit(new Committed(commitHydrants, commitMetadata)); + commitPartialSegments.putAll(currentPartialSegments); + writeCommit(new Committed(commitPartialSegments, commitMetadata)); } finally { commitLock.unlock(); @@ -723,12 +726,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 segment: [%d], total partial segment (across appendable segment): [%d], persisted partial segment (across appendable segment): [%d]", rowIngestionMeters.getProcessed(), totalPersistedRows.get(), - totalSinks, - totalHydrantsCount.longValue(), - totalHydrantsPersisted.longValue() + totalAppendableSegments, + totalPartialSegmentsCount.longValue(), + totalPartialSegmentsPersisted.longValue() ); // return null if committer is null @@ -771,19 +774,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( @@ -794,18 +797,18 @@ 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 segment (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; } @@ -859,23 +862,23 @@ 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 + * @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 sink is no longer valid + * @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; } @@ -885,14 +888,14 @@ 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); } } } @@ -932,10 +935,10 @@ private DataSegmentWithMetadata mergeAndPush( List indexes = new ArrayList<>(); 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(); - 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); } @@ -964,7 +967,7 @@ private DataSegmentWithMetadata mergeAndPush( closer.close(); } - final DataSegment segmentToPush = sink.getSegment().withDimensions( + final DataSegment segmentToPush = appendableSegment.getSegment().withDimensions( IndexMerger.getMergedDimensionsFromQueryableIndexes(indexes, schema.getDimensionsSpec()) ); @@ -1013,7 +1016,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)); } @@ -1054,7 +1057,7 @@ public void close() // Only unlock if executors actually shut down. unlockBasePersistDirectory(); - sinkSchemaAnnouncer.stop(); + appendableSegmentSchemaAnnouncer.stop(); } /** @@ -1074,7 +1077,7 @@ public void closeNow() } log.debug("Shutting down immediately..."); - for (Map.Entry entry : sinks.entrySet()) { + for (Map.Entry entry : appendableSegments.entrySet()) { unannounceAllVersionsOfSegment(entry.getValue().getSegment(), entry.getValue()); } try { @@ -1096,21 +1099,21 @@ public void closeNow() throw new ISE("Failed to shutdown executors during close()"); } - sinkSchemaAnnouncer.stop(); + appendableSegmentSchemaAnnouncer.stop(); } @VisibleForTesting - SinkSchemaAnnouncer getSinkSchemaAnnouncer() + AppendableSegmentSchemaAnnouncer getAppendableSegmentSchemaAnnouncer() { - return sinkSchemaAnnouncer; + return appendableSegmentSchemaAnnouncer; } /** * Unannounces the given base segment and all its upgraded versions. */ - private void unannounceAllVersionsOfSegment(DataSegment baseSegment, Sink sink) + private void unannounceAllVersionsOfSegment(DataSegment baseSegment, AppendableSegment appendableSegment) { - synchronized (sink) { + synchronized (appendableSegment) { final SegmentIdWithShardSpec baseId = SegmentIdWithShardSpec.fromDataSegment(baseSegment); if (!baseSegmentToUpgradedSegments.containsKey(baseId)) { return; @@ -1151,15 +1154,15 @@ public void registerUpgradedPendingSegment(PendingSegmentRecord pendingSegmentRe { SegmentIdWithShardSpec basePendingSegment = idToPendingSegment.get(pendingSegmentRecord.getUpgradedFromSegmentId()); SegmentIdWithShardSpec upgradedPendingSegment = pendingSegmentRecord.getId(); - if (!sinks.containsKey(basePendingSegment) || droppingSinks.contains(basePendingSegment)) { + if (!appendableSegments.containsKey(basePendingSegment) || droppingAppendableSegments.contains(basePendingSegment)) { return; } - // Update query mapping with SinkQuerySegmentWalker - ((SinkQuerySegmentWalker) texasRanger).registerUpgradedPendingSegment(basePendingSegment, upgradedPendingSegment); + // Update query mapping with AppendableSegmentQuerySegmentWalker + ((AppendableSegmentQuerySegmentWalker) texasRanger).registerUpgradedPendingSegment(basePendingSegment, upgradedPendingSegment); // Announce segments - final DataSegment baseSegment = sinks.get(basePendingSegment).getSegment(); + final DataSegment baseSegment = appendableSegments.get(basePendingSegment).getSegment(); final DataSegment newSegment = getUpgradedSegment(baseSegment, upgradedPendingSegment); segmentAnnouncer.announceSegment(newSegment); @@ -1273,13 +1276,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()) { @@ -1317,71 +1321,71 @@ 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(), @@ -1390,27 +1394,27 @@ private Object bootstrapSinksFromDisk() tuningConfig.getMaxRowsInMemory(), maxBytesTuningConfig, useMaxMemoryEstimates, - hydrants + partialSegments ); - rowsSoFar += currSink.getNumRows(); + rowsSoFar += currAppendableSegment.getNumRows(); - addSink(identifier, currSink); - segmentAnnouncer.announceSegment(currSink.getSegment()); + addAppendableSegment(identifier, currAppendableSegment); + 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 missingSinks = Sets.difference(committed.getHydrants().keySet(), loadedSinks); - if (!missingSinks.isEmpty()) { - throw new ISE("Missing committed sinks [%s]", Joiner.on(", ").join(missingSinks)); + final Set missingAppendableSegments = Sets.difference(committed.getPartialSegments().keySet(), loadedAppendableSegments); + if (!missingAppendableSegments.isEmpty()) { + throw new ISE("Missing committed appendable segments [%s]", Joiner.on(", ").join(missingAppendableSegments)); } totalRows.set(rowsSoFar); @@ -1418,66 +1422,67 @@ private Object bootstrapSinksFromDisk() } /** - * Update the state of the appenderator when adding a sink. + * Update the state of the appenderator when adding a appendable segment. * - * @param identifier sink identifier - * @param sink sink to be added + * @param identifier appendable segment identifier + * @param appendableSegment appendable segment to be added */ - private void addSink(SegmentIdWithShardSpec identifier, Sink sink) + private void addAppendableSegment(SegmentIdWithShardSpec identifier, AppendableSegment appendableSegment) { - sinks.put(identifier, sink); - // Asoociate the base segment of a sink with its string identifier + appendableSegments.put(identifier, appendableSegment); + // Asoociate the base segment of a appendable segment with its string identifier // Needed to get the base segment using upgradedFromSegmentId of a pending segment idToPendingSegment.put(identifier.asSegmentId().toString(), identifier); - // The base segment is associated with itself in the maps to maintain all the upgraded ids of a sink. + // The base segment is associated with itself in the maps to maintain all the upgraded ids of a appendable segment. baseSegmentToUpgradedSegments.put(identifier, new HashSet<>()); baseSegmentToUpgradedSegments.get(identifier).add(identifier); - sinkTimeline.add( - sink.getInterval(), - sink.getVersion(), - identifier.getShardSpec().createChunk(sink) + appendableSegmentTimeline.add( + appendableSegment.getInterval(), + appendableSegment.getVersion(), + identifier.getShardSpec().createChunk(appendableSegment) ); } private ListenableFuture abandonSegment( final SegmentIdWithShardSpec identifier, - final Sink sink, + final AppendableSegment appendableSegment, final boolean removeOnDiskData ) { abandonedSegments.add(identifier); final SegmentIdWithShardSpec baseIdentifier = upgradedSegmentToBaseSegment.getOrDefault(identifier, identifier); - synchronized (sink) { + synchronized (appendableSegment) { if (baseSegmentToUpgradedSegments.containsKey(baseIdentifier)) { Set relevantSegments = new HashSet<>(baseSegmentToUpgradedSegments.get(baseIdentifier)); relevantSegments.removeAll(abandonedSegments); - // If there are unabandoned segments associated with the sink, return early + // If there are unabandoned segments associated with the appendable segment, return early // This may be the case if segments have been upgraded as the result of a concurrent replace if (!relevantSegments.isEmpty()) { return Futures.immediateFuture(null); } } } - // 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(sink)); - 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 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.addAndGet(-appendableSegment.getNumRowsInMemory()); + bytesCurrentlyInMemory.addAndGet(-appendableSegment.getBytesInMemory()); + bytesCurrentlyInMemory.addAndGet(-calculateAppendableSegmentMemoryInUsed(appendableSegment)); + for (PartialSegment partialSegment : appendableSegment) { + // Decrement memory used by all Memory Mapped partial segment + if (!partialSegment.equals(appendableSegment.getCurrentPartialSegment())) { + bytesCurrentlyInMemory.addAndGet(-calculateMMappedPartialSegmentsMemoryInUsed(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(baseIdentifier); + droppingAppendableSegments.add(baseIdentifier); // Wait for any outstanding pushes to finish, then abandon the segment inside the persist thread. return Futures.transform( @@ -1488,12 +1493,12 @@ private ListenableFuture abandonSegment( @Override public Void apply(@Nullable Object input) { - if (!sinks.remove(baseIdentifier, sink)) { - log.error("Sink for segment[%s] no longer valid, not abandoning.", baseIdentifier); + if (!appendableSegments.remove(baseIdentifier, appendableSegment)) { + log.error("Appendable segment[%s] no longer valid, not abandoning.", baseIdentifier); 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. @@ -1516,20 +1521,20 @@ public Void apply(@Nullable Object input) } } - unannounceAllVersionsOfSegment(sink.getSegment(), sink); + unannounceAllVersionsOfSegment(appendableSegment.getSegment(), appendableSegment); Runnable removeRunnable = () -> { - droppingSinks.remove(baseIdentifier); - sinkTimeline.remove( - sink.getInterval(), - sink.getVersion(), - baseIdentifier.getShardSpec().createChunk(sink) + droppingAppendableSegments.remove(baseIdentifier); + appendableSegmentTimeline.remove( + appendableSegment.getInterval(), + appendableSegment.getVersion(), + baseIdentifier.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); + partialSegment.swapSegment(null); } if (removeOnDiskData) { @@ -1551,9 +1556,9 @@ public Void apply(@Nullable Object input) identifier, segmentLoaderConfig.getDropSegmentDelayMillis() ); - // Keep the segments in the cache and sinkTimeline for dropSegmentDelay after unannouncing the segments - // This way, in transit queries which still see the segments in this peon would be able to query the - // segments and not throw NullPtr exceptions. + // Keep the segments in the cache and appendableSegmentTimeline for dropSegmentDelay after unannouncing + // the segments. This way, in transit queries which still see the segments in this peon would be able to + // query the segments and not throw NullPtr exceptions. exec.schedule( removeRunnable, segmentLoaderConfig.getDropSegmentDelayMillis(), @@ -1574,7 +1579,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; @@ -1623,27 +1628,27 @@ 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. + * Persists the given {@link PartialSegment} 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 + * @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) { synchronized (indexToPersist) { if (indexToPersist.hasSwapped()) { log.info( - "Segment[%s] hydrant[%s] already swapped. Ignoring request to persist.", + "Segment[%s] partial segment[%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 partial segment[%s]", identifier, indexToPersist); try { final long startTime = System.nanoTime(); @@ -1700,36 +1705,36 @@ private void removeDirectory(final File target) } } - private int calculateMMappedHydrantMemoryInUsed(FireHydrant hydrant) + private int calculateMMappedPartialSegmentsMemoryInUsed(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 partial segment, shorts in ReferenceCountingSegment, // Objects in SimpleQueryableIndex (such as SmooshedFileMapper, each ColumnHolder in column map, etc.) - int total = Integer.BYTES + (4 * Short.BYTES) + ROUGH_OVERHEAD_PER_HYDRANT + - (hydrant.getSegmentNumDimensionColumns() * ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER) + - (hydrant.getSegmentNumMetricColumns() * ROUGH_OVERHEAD_PER_METRIC_COLUMN_HOLDER) + + int total = Integer.BYTES + (4 * Short.BYTES) + ROUGH_OVERHEAD_PER_PARTIAL_SEGMENT + + (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(Sink sink) + private int calculateAppendableSegmentMemoryInUsed(AppendableSegment appendableSegment) { 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 inner class periodically computes absolute and delta schema for all the {@link StreamAppenderator#sinks} + * This inner class periodically computes absolute and delta schema for all the {@link StreamAppenderator#appendableSegments} * and announces them. */ @VisibleForTesting - class SinkSchemaAnnouncer + class AppendableSegmentSchemaAnnouncer { private static final long SCHEMA_PUBLISH_DELAY_MILLIS = 0; private static final long SCHEMA_PUBLISH_PERIOD_MILLIS = 60_000; @@ -1737,15 +1742,15 @@ class SinkSchemaAnnouncer private final DataSegmentAnnouncer announcer; private final ScheduledExecutorService scheduledExecutorService; private final String taskId; - // This structure is accessed only by a single thread (Sink-Schema-Announcer-0), hence it is not thread safe. - private Map> previousSinkSignatureMap = new HashMap<>(); + // This structure is accessed only by a single thread (Appendable-Segment-Schema-Announcer-0), hence it is not thread safe. + private Map> previousAppendableSegmentsSignatureMap = new HashMap<>(); - SinkSchemaAnnouncer() + AppendableSegmentSchemaAnnouncer() { this.announcer = StreamAppenderator.this.segmentAnnouncer; this.taskId = StreamAppenderator.this.myId; boolean enabled = centralizedDatasourceSchemaConfig.isEnabled(); - this.scheduledExecutorService = enabled ? ScheduledExecutors.fixed(1, "Sink-Schema-Announcer-%d") : null; + this.scheduledExecutorService = enabled ? ScheduledExecutors.fixed(1, "Appendable-Segment-Schema-Announcer-%d") : null; } private void start() @@ -1771,25 +1776,28 @@ private void stop() @VisibleForTesting void computeAndAnnounce() { - Map> currentSinkSignatureMap = new HashMap<>(); - for (Map.Entry sinkEntry : StreamAppenderator.this.sinks.entrySet()) { - SegmentIdWithShardSpec segmentIdWithShardSpec = sinkEntry.getKey(); - Sink sink = sinkEntry.getValue(); - currentSinkSignatureMap.put(segmentIdWithShardSpec.asSegmentId(), Pair.of(sink.getSignature(), sink.getNumRows())); + Map> currentAppendableSegmentSignatureMap = new HashMap<>(); + for (Map.Entry appendableSegmentEntry : StreamAppenderator.this.appendableSegments.entrySet()) { + SegmentIdWithShardSpec segmentIdWithShardSpec = appendableSegmentEntry.getKey(); + AppendableSegment appendableSegment = appendableSegmentEntry.getValue(); + currentAppendableSegmentSignatureMap.put(segmentIdWithShardSpec.asSegmentId(), Pair.of(appendableSegment.getSignature(), appendableSegment.getNumRows())); } - Optional sinksSchema = SinkSchemaUtil.computeAbsoluteSchema(currentSinkSignatureMap); - Optional sinksSchemaChange = SinkSchemaUtil.computeSchemaChange(previousSinkSignatureMap, currentSinkSignatureMap); + Optional appendableSegmentsSchema = AppendableSegmentSchemaUtil.computeAbsoluteSchema(currentAppendableSegmentSignatureMap); + Optional appendableSegmentsSchemaChange = AppendableSegmentSchemaUtil.computeSchemaChange( + previousAppendableSegmentsSignatureMap, + currentAppendableSegmentSignatureMap + ); // update previous reference - previousSinkSignatureMap = currentSinkSignatureMap; + previousAppendableSegmentsSignatureMap = currentAppendableSegmentSignatureMap; // announce schema - sinksSchema.ifPresent( + appendableSegmentsSchema.ifPresent( segmentsSchema -> announcer.announceSegmentSchemas( taskId, segmentsSchema, - sinksSchemaChange.orElse(null) + appendableSegmentsSchemaChange.orElse(null) )); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java index ffdfb8d1eb07..a9b2f000903b 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java @@ -64,8 +64,8 @@ import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.AppendableSegment; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; -import org.apache.druid.segment.realtime.sink.Sink; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.timeline.VersionedIntervalTimeline; @@ -84,16 +84,16 @@ * a single process. * * This class keeps a map of {@link DatasourceBundle} objects, keyed by datasource name. Each bundle contains: - * - A per-datasource {@link SinkQuerySegmentWalker} (with an associated per-datasource timeline) + * - A per-datasource {@link AppendableSegmentQuerySegmentWalker} (with an associated per-datasource timeline) * - A map that associates a taskId with a list of Appenderators created for that task * * Access to the datasource bundle map and the task->appenderator maps is synchronized. The methods * on this class can be called concurrently from multiple task threads. If there are no remaining * appenderators for a given datasource, the corresponding bundle will be removed from the bundle map. * - * Appenderators created by this class will use the shared per-datasource SinkQuerySegmentWalkers. + * Appenderators created by this class will use the shared per-datasource {@link AppendableSegmentQuerySegmentWalker). * - * The per-datasource SinkQuerySegmentWalkers share a common queryExecutorService. + * The per-datasource {@link AppendableSegmentQuerySegmentWalker} share a common queryExecutorService. * * Each task that requests an Appenderator from this AppenderatorsManager will receive a heap memory limit * equal to {@link WorkerConfig#globalIngestionHeapLimitBytes} evenly divided by {@link WorkerConfig#capacity}. @@ -426,7 +426,7 @@ private AppenderatorConfig rewriteAppenderatorConfigMemoryLimits(AppenderatorCon @VisibleForTesting public class DatasourceBundle { - private final SinkQuerySegmentWalker walker; + private final AppendableSegmentQuerySegmentWalker walker; private final Map> taskAppenderatorMap; public DatasourceBundle( @@ -435,10 +435,10 @@ public DatasourceBundle( { this.taskAppenderatorMap = new HashMap<>(); - VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline<>( + VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline<>( String.CASE_INSENSITIVE_ORDER ); - this.walker = new SinkQuerySegmentWalker( + this.walker = new AppendableSegmentQuerySegmentWalker( dataSource, sinkTimeline, objectMapper, @@ -451,7 +451,7 @@ public DatasourceBundle( ); } - public SinkQuerySegmentWalker getWalker() + public AppendableSegmentQuerySegmentWalker getWalker() { return walker; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ClippedFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/ClippedFirehoseFactory.java deleted file mode 100644 index 9ca595ce5c28..000000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ClippedFirehoseFactory.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.segment.realtime.firehose; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Predicate; -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.joda.time.Interval; - -import java.io.File; -import java.io.IOException; - -/** - * Creates firehoses clipped to a particular time interval. Useful for enforcing min time, max time, and time windows. - */ -@Deprecated -public class ClippedFirehoseFactory implements FirehoseFactory -{ - private final FirehoseFactory delegate; - private final Interval interval; - - @JsonCreator - public ClippedFirehoseFactory( - @JsonProperty("delegate") FirehoseFactory delegate, - @JsonProperty("interval") Interval interval - ) - { - this.delegate = delegate; - this.interval = interval; - } - - @JsonProperty - public FirehoseFactory getDelegate() - { - return delegate; - } - - @JsonProperty - public Interval getInterval() - { - return interval; - } - - @Override - public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException - { - return new PredicateFirehose( - delegate.connect(parser, temporaryDirectory), - new Predicate() - { - @Override - public boolean apply(InputRow input) - { - return interval.contains(input.getTimestampFromEpoch()); - } - } - ); - } - -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiver.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiver.java deleted file mode 100644 index e2b776fa4fa5..000000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiver.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.segment.realtime.firehose; - -import java.util.Collection; -import java.util.Map; - -public interface EventReceiver -{ - void addAll(Collection> events); -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java deleted file mode 100644 index 4fad62a6d462..000000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java +++ /dev/null @@ -1,659 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.segment.realtime.firehose; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; -import com.google.common.io.CountingInputStream; -import com.google.common.util.concurrent.Uninterruptibles; -import com.google.errorprone.annotations.concurrent.GuardedBy; -import org.apache.druid.concurrent.Threads; -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.guice.annotations.Json; -import org.apache.druid.guice.annotations.Smile; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.server.metrics.EventReceiverFirehoseMetric; -import org.apache.druid.server.metrics.EventReceiverFirehoseRegister; -import org.apache.druid.server.security.Access; -import org.apache.druid.server.security.Action; -import org.apache.druid.server.security.AuthorizationUtils; -import org.apache.druid.server.security.AuthorizerMapper; -import org.apache.druid.server.security.Resource; -import org.apache.druid.server.security.ResourceAction; -import org.joda.time.DateTime; - -import javax.annotation.Nullable; -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.Consumes; -import javax.ws.rs.POST; -import javax.ws.rs.Path; -import javax.ws.rs.Produces; -import javax.ws.rs.QueryParam; -import javax.ws.rs.core.Context; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; - -/** - * Builds firehoses that accept events through the {@link EventReceiver} interface. Can also register these - * firehoses with an {@link ServiceAnnouncingChatHandlerProvider}. - */ -@Deprecated -public class EventReceiverFirehoseFactory implements FirehoseFactory>> -{ - private static final EmittingLogger log = new EmittingLogger(EventReceiverFirehoseFactory.class); - - public static final int MAX_FIREHOSE_PRODUCERS = 10_000; - - private static final int DEFAULT_BUFFER_SIZE = 100_000; - - /** - * A "poison pill" object for {@link EventReceiverFirehose}'s internal buffer. - */ - private static final Object FIREHOSE_CLOSED = new Object(); - - private final String serviceName; - private final int bufferSize; - - /** - * Doesn't really support max idle times finer than 1 second due to how {@link - * EventReceiverFirehose#delayedCloseExecutor} is implemented, see a comment inside {@link - * EventReceiverFirehose#createDelayedCloseExecutor()}. This aspect is not reflected in docs because it's unlikely - * that anybody configures or cares about finer max idle times, and also because this is an implementation detail of - * {@link EventReceiverFirehose} that may change in the future. - */ - private final long maxIdleTimeMillis; - private final ChatHandlerProvider chatHandlerProvider; - private final ObjectMapper jsonMapper; - private final ObjectMapper smileMapper; - private final EventReceiverFirehoseRegister eventReceiverFirehoseRegister; - private final AuthorizerMapper authorizerMapper; - - @JsonCreator - public EventReceiverFirehoseFactory( - @JsonProperty("serviceName") String serviceName, - @JsonProperty("bufferSize") Integer bufferSize, - // Keeping the legacy 'maxIdleTime' property name for backward compatibility. When the project is updated to - // Jackson 2.9 it could be changed, see https://github.com/apache/druid/issues/7152 - @JsonProperty("maxIdleTime") @Nullable Long maxIdleTimeMillis, - @JacksonInject ChatHandlerProvider chatHandlerProvider, - @JacksonInject @Json ObjectMapper jsonMapper, - @JacksonInject @Smile ObjectMapper smileMapper, - @JacksonInject EventReceiverFirehoseRegister eventReceiverFirehoseRegister, - @JacksonInject AuthorizerMapper authorizerMapper - ) - { - Preconditions.checkNotNull(serviceName, "serviceName"); - - this.serviceName = serviceName; - this.bufferSize = bufferSize == null || bufferSize <= 0 ? DEFAULT_BUFFER_SIZE : bufferSize; - this.maxIdleTimeMillis = (maxIdleTimeMillis == null || maxIdleTimeMillis <= 0) ? Long.MAX_VALUE : maxIdleTimeMillis; - this.chatHandlerProvider = chatHandlerProvider; - this.jsonMapper = jsonMapper; - this.smileMapper = smileMapper; - this.eventReceiverFirehoseRegister = eventReceiverFirehoseRegister; - this.authorizerMapper = authorizerMapper; - } - - @Override - public Firehose connect( - InputRowParser> firehoseParser, - File temporaryDirectory - ) - { - log.info("Connecting firehose: %s", serviceName); - final EventReceiverFirehose firehose = new EventReceiverFirehose(firehoseParser); - - if (chatHandlerProvider != null) { - log.info("Found chathandler of class[%s]", chatHandlerProvider.getClass().getName()); - chatHandlerProvider.register(serviceName, firehose); - int lastIndexOfColon = serviceName.lastIndexOf(':'); - if (lastIndexOfColon > 0) { - chatHandlerProvider.register(serviceName.substring(lastIndexOfColon + 1), firehose); - } - } else { - log.warn("No chathandler detected"); - } - - eventReceiverFirehoseRegister.register(serviceName, firehose); - - return firehose; - } - - @JsonProperty - public String getServiceName() - { - return serviceName; - } - - @JsonProperty - public int getBufferSize() - { - return bufferSize; - } - - /** - * Keeping the legacy 'maxIdleTime' property name for backward compatibility. When the project is updated to Jackson - * 2.9 it could be changed, see https://github.com/apache/druid/issues/7152 - */ - @JsonProperty("maxIdleTime") - public long getMaxIdleTimeMillis() - { - return maxIdleTimeMillis; - } - - /** - * Apart from adhering to {@link Firehose} contract regarding concurrency, this class has two methods that might be - * called concurrently with any other methods and each other, from arbitrary number of threads: {@link #addAll} and - * {@link #shutdown}. - * - * Concurrent data flow: in {@link #addAll} (can be called concurrently with any other methods and other calls to - * {@link #addAll}) rows are pushed into {@link #buffer}. The single Firehose "consumer" thread calls {@link #hasMore} - * and {@link #nextRow()}, where rows are taken out from the other end of the {@link #buffer} queue. - * - * This class creates and manages one thread ({@link #delayedCloseExecutor}) for calling {@link #close()} - * asynchronously in response to a {@link #shutdown} request, or after this Firehose has been idle (no calls to {@link - * #addAll}) for {@link #maxIdleTimeMillis}. - */ - @VisibleForTesting - public class EventReceiverFirehose implements ChatHandler, Firehose, EventReceiverFirehoseMetric - { - /** - * How does this thread work (and its interruption policy) is described in the comment for {@link - * #createDelayedCloseExecutor}. - */ - @GuardedBy("this") - private @Nullable Thread delayedCloseExecutor; - - /** - * Contains {@link InputRow} objects, the last one is {@link #FIREHOSE_CLOSED} which is a "poison pill". Poison pill - * is used to notify the thread that calls {@link #hasMore()} and {@link #nextRow()} that the EventReceiverFirehose - * is closed without heuristic 500 ms timed blocking in a loop instead of a simple {@link BlockingQueue#take()} - * call (see {@link #hasMore} code). - */ - private final BlockingQueue buffer; - private final InputRowParser> parser; - - /** - * This field needs to be volatile to ensure progress in {@link #addRows} method where it is read in a loop, and - * also in testing code calling {@link #isClosed()}. - */ - private volatile boolean closed = false; - - /** - * This field and {@link #rowsRunOut} are not volatile because they are accessed only from {@link #hasMore()} and - * {@link #nextRow()} methods that are called from a single thread according to {@link Firehose} spec. - */ - @Nullable - private InputRow nextRow = null; - private boolean rowsRunOut = false; - - private final AtomicLong bytesReceived = new AtomicLong(0); - private final AtomicLong lastBufferAddFailLoggingTimeNs = new AtomicLong(System.nanoTime()); - private final ConcurrentHashMap producerSequences = new ConcurrentHashMap<>(); - - /** - * This field and {@link #requestedShutdownTimeNs} use nanoseconds instead of milliseconds not to deal with the fact - * that {@link System#currentTimeMillis()} can "go backward", e. g. due to time correction on the server. - * - * This field and {@link #requestedShutdownTimeNs} must be volatile because they are de facto lazily initialized - * fields that are used concurrently in {@link #delayedCloseExecutor} (see {@link #createDelayedCloseExecutor()}). - * If they were not volatile, NPE would be possible in {@link #delayedCloseExecutor}. See - * https://shipilev.net/blog/2016/close-encounters-of-jmm-kind/#wishful-hb-actual for explanations. - */ - @Nullable - private volatile Long idleCloseTimeNs = null; - @Nullable - private volatile Long requestedShutdownTimeNs = null; - - EventReceiverFirehose(InputRowParser> parser) - { - this.buffer = new ArrayBlockingQueue<>(bufferSize); - this.parser = parser; - - if (maxIdleTimeMillis != Long.MAX_VALUE) { - idleCloseTimeNs = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(maxIdleTimeMillis); - synchronized (this) { - createDelayedCloseExecutor(); - } - } - } - - @VisibleForTesting - synchronized @Nullable Thread getDelayedCloseExecutor() - { - return delayedCloseExecutor; - } - - /** - * Creates and starts a {@link #delayedCloseExecutor} thread, either right from the EventReceiverFirehose's - * constructor if {@link #maxIdleTimeMillis} is specified, or otherwise lazily from {@link #shutdown}. - * - * The thread waits until the time when the Firehose should be closed because either {@link #addAll} was not called - * for the specified max idle time (see {@link #idleCloseTimeNs}), or until the shutoff time requested last - * via {@link #shutdown} (see {@link #requestedShutdownTimeNs}), whatever is sooner. Then the thread does - * two things: - * 1. if the Firehose is already closed (or in the process of closing, but {@link #closed} flag is already set), it - * silently exits. - * 2. It checks both deadlines again: - * a) if either of them has arrived, it calls {@link #close()} and exits. - * b) otherwise, it waits until the nearest deadline again, and so on in a loop. - * - * This way the thread works predictably and robustly regardless of how both deadlines change (for example, shutoff - * time specified via {@link #shutdown} may jump in both directions). - * - * Other methods notify {@link #delayedCloseExecutor} that the Firehose state in some way that is important for this - * thread (that is, when {@link #close()} is called, {@link #delayedCloseExecutor} is no longer needed and should - * exit as soon as possible to release system resources; when {@link #shutdown} is called, the thread may need to - * wake up sooner if the shutoff time has been moved sooner) by simply interrupting it. The thread wakes up and - * continues its loop. - */ - @GuardedBy("this") - private Thread createDelayedCloseExecutor() - { - Thread delayedCloseExecutor = new Thread( - () -> { - // The closed = true is visible after close() because there is a happens-before edge between - // delayedCloseExecutor.interrupt() call in close() and catching InterruptedException below in this loop. - while (!closed) { - if (idleCloseTimeNs == null && requestedShutdownTimeNs == null) { - // This is not possible unless there are bugs in the code of EventReceiverFirehose. AssertionError could - // have been thrown instead, but it doesn't seem to make a lot of sense in a background thread. Instead, - // we long the error and continue a loop after some pause. - log.error( - "Either idleCloseTimeNs or requestedShutdownTimeNs must be non-null. " - + "Please file a bug at https://github.com/apache/druid/issues" - ); - } - if (idleCloseTimeNs != null && idleCloseTimeNs - System.nanoTime() <= 0) { // overflow-aware comparison - log.info("Firehose has been idle for %d ms, closing.", maxIdleTimeMillis); - close(); - } else if (requestedShutdownTimeNs != null && - requestedShutdownTimeNs - System.nanoTime() <= 0) { // overflow-aware comparison - log.info("Closing Firehose after a shutdown request"); - close(); - } - try { - // It is possible to write code that sleeps until the next the next idleCloseTimeNs or - // requestedShutdownTimeNs, whatever is non-null and sooner, but that's fairly complicated code. That - // complexity perhaps overweighs the minor inefficiency of simply waking up every second. - Threads.sleepFor(1, TimeUnit.SECONDS); - } - catch (InterruptedException ignore) { - // Interruption is a wakeup, continue the loop - } - } - }, - "event-receiver-firehose-closer" - ); - delayedCloseExecutor.setDaemon(true); - this.delayedCloseExecutor = delayedCloseExecutor; - delayedCloseExecutor.start(); - return delayedCloseExecutor; - } - - /** - * This method might be called concurrently from multiple threads, if multiple requests arrive to the server at the - * same time (possibly exact duplicates). Concurrency is controlled in {@link #checkProducerSequence}, where only - * requests with "X-Firehose-Producer-Seq" number greater than the max "X-Firehose-Producer-Seq" in previously - * arrived requests are allowed to proceed. After that check requests don't synchronize with each other and - * therefore if two large batches are sent with little interval, the events from the batches might be mixed up in - * {@link #buffer} (if two {@link #addRows(Iterable)} are executed concurrently). - */ - @POST - @Path("/push-events") - @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) - @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) - public Response addAll(InputStream in, @Context final HttpServletRequest req) throws JsonProcessingException - { - idleCloseTimeNs = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(maxIdleTimeMillis); - Access accessResult = AuthorizationUtils.authorizeResourceAction( - req, - new ResourceAction( - Resource.STATE_RESOURCE, - Action.WRITE - ), - authorizerMapper - ); - if (!accessResult.isAllowed()) { - return Response.status(403).build(); - } - - final String reqContentType = req.getContentType(); - final boolean isSmile = SmileMediaTypes.APPLICATION_JACKSON_SMILE.equals(reqContentType); - final String contentType = isSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON; - - ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper; - - Response producerSequenceResponse = checkProducerSequence(req, reqContentType, objectMapper); - if (producerSequenceResponse != null) { - return producerSequenceResponse; - } - - CountingInputStream countingInputStream = new CountingInputStream(in); - Collection> events; - try { - events = objectMapper.readValue( - countingInputStream, - new TypeReference>>() - { - } - ); - } - catch (IOException e) { - return Response.serverError().entity(ImmutableMap.of("error", e.getMessage())).build(); - } - finally { - bytesReceived.addAndGet(countingInputStream.getCount()); - } - log.debug("Adding %,d events to firehose: %s", events.size(), serviceName); - - final List rows = new ArrayList<>(); - for (final Map event : events) { - // Might throw an exception. We'd like that to happen now, instead of while adding to the row buffer. - rows.addAll(parser.parseBatch(event)); - } - - try { - addRows(rows); - return Response.ok( - objectMapper.writeValueAsString(ImmutableMap.of("eventCount", events.size())), - contentType - ).build(); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - } - - @Override - public boolean hasMore() - { - if (rowsRunOut) { - return false; - } - if (nextRow != null) { - return true; - } - Object next; - try { - next = buffer.take(); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - //noinspection ObjectEquality - if (next == FIREHOSE_CLOSED) { - rowsRunOut = true; - return false; - } - nextRow = (InputRow) next; - return true; - } - - @Nullable - @Override - public InputRow nextRow() - { - final InputRow row = nextRow; - - if (row == null) { - throw new NoSuchElementException(); - } else { - nextRow = null; - return row; - } - } - - @Override - public int getCurrentBufferSize() - { - return buffer.size(); - } - - @Override - public int getCapacity() - { - return bufferSize; - } - - @Override - public long getBytesReceived() - { - return bytesReceived.get(); - } - - /** - * This method is synchronized because it might be called concurrently from multiple threads: from {@link - * #delayedCloseExecutor}, and from the thread that creates and uses the Firehose object. - */ - @Override - public synchronized void close() - { - if (closed) { - return; - } - closed = true; - log.info("Firehose closing."); - - // Critical to add the poison pill to the queue, don't allow interruption. - Uninterruptibles.putUninterruptibly(buffer, FIREHOSE_CLOSED); - - eventReceiverFirehoseRegister.unregister(serviceName); - if (chatHandlerProvider != null) { - chatHandlerProvider.unregister(serviceName); - } - if (delayedCloseExecutor != null && !delayedCloseExecutor.equals(Thread.currentThread())) { - // Interrupt delayedCloseExecutor to let it discover that closed flag is already set and exit. - delayedCloseExecutor.interrupt(); - } - } - - @VisibleForTesting - void addRows(Iterable rows) throws InterruptedException - { - for (final InputRow row : rows) { - boolean added = false; - while (!closed && !added) { - added = buffer.offer(row, 500, TimeUnit.MILLISECONDS); - if (!added) { - long currTimeNs = System.nanoTime(); - long lastTimeNs = lastBufferAddFailLoggingTimeNs.get(); - if (currTimeNs - lastTimeNs > TimeUnit.SECONDS.toNanos(10) && - lastBufferAddFailLoggingTimeNs.compareAndSet(lastTimeNs, currTimeNs)) { - log.warn("Failed to add event to buffer with current size [%s] . Retrying...", buffer.size()); - } - } - } - - if (!added) { - throw new IllegalStateException("Cannot add events to closed firehose!"); - } - } - } - - /** - * This method might be called concurrently from multiple threads, if multiple shutdown requests arrive at the same - * time. No attempts are made to synchronize such requests, or prioritize them a-la "latest shutdown time wins" or - * "soonest shutdown time wins". {@link #delayedCloseExecutor}'s logic (see {@link #createDelayedCloseExecutor()}) - * is indifferent to shutdown times jumping in arbitrary directions. But once a shutdown request is made, it can't - * be cancelled entirely, the shutdown time could only be rescheduled with a new request. - */ - @POST - @Path("/shutdown") - @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) - @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) - public Response shutdown( - @QueryParam("shutoffTime") final String shutoffTimeMillis, - @Context final HttpServletRequest req - ) - { - Access accessResult = AuthorizationUtils.authorizeResourceAction( - req, - new ResourceAction( - Resource.STATE_RESOURCE, - Action.WRITE - ), - authorizerMapper - ); - if (!accessResult.isAllowed()) { - return Response.status(403).build(); - } - - try { - DateTime shutoffAt = shutoffTimeMillis == null ? DateTimes.nowUtc() : DateTimes.of(shutoffTimeMillis); - log.info("Setting Firehose shutoffTime to %s", shutoffTimeMillis); - long shutoffTimeoutMillis = Math.max(shutoffAt.getMillis() - System.currentTimeMillis(), 0); - - requestedShutdownTimeNs = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(shutoffTimeoutMillis); - Thread delayedCloseExecutor; - // Need to interrupt delayedCloseExecutor because a newly specified shutdown time might be closer than idle - // timeout or previously specified shutdown. Interruption of delayedCloseExecutor lets it adjust the sleep time - // (see the logic of this thread in createDelayedCloseExecutor()). - boolean needToInterruptDelayedCloseExecutor = true; - synchronized (this) { - delayedCloseExecutor = this.delayedCloseExecutor; - if (delayedCloseExecutor == null) { - delayedCloseExecutor = createDelayedCloseExecutor(); - // Don't need to interrupt a freshly created thread - needToInterruptDelayedCloseExecutor = false; - } - } - if (needToInterruptDelayedCloseExecutor) { - delayedCloseExecutor.interrupt(); - } - return Response.ok().build(); - } - catch (IllegalArgumentException e) { - return Response.status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of("error", e.getMessage())) - .build(); - - } - } - - @VisibleForTesting - boolean isClosed() - { - return closed; - } - - /** - * Checks the request for a producer ID and sequence value. If the producer ID is specified, a corresponding - * sequence value must be specified as well. If the incoming sequence is less than or equal to the last seen - * sequence for that producer ID, the request is ignored. - * - * This method might be called concurrently from multiple threads. - * - * @param req Http request - * @param responseContentType Response content type - * @param responseMapper Response object mapper - * @return an error response to return or null if the request can proceed - */ - @Nullable - private Response checkProducerSequence( - final HttpServletRequest req, - final String responseContentType, - final ObjectMapper responseMapper - ) - { - final String producerId = req.getHeader("X-Firehose-Producer-Id"); - - if (producerId == null) { - return null; - } - - final String sequenceValue = req.getHeader("X-Firehose-Producer-Seq"); - - if (sequenceValue == null) { - return Response - .status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of("error", "Producer sequence value is missing")) - .build(); - } - - Long producerSequence = producerSequences.computeIfAbsent(producerId, key -> Long.MIN_VALUE); - - if (producerSequences.size() >= MAX_FIREHOSE_PRODUCERS) { - return Response - .status(Response.Status.FORBIDDEN) - .entity( - ImmutableMap.of( - "error", - "Too many individual producer IDs for this firehose. Max is " + MAX_FIREHOSE_PRODUCERS - ) - ) - .build(); - } - - try { - Long newSequence = Long.parseLong(sequenceValue); - - while (true) { - if (newSequence <= producerSequence) { - return Response.ok( - responseMapper.writeValueAsString(ImmutableMap.of("eventCount", 0, "skipped", true)), - responseContentType - ).build(); - } - if (producerSequences.replace(producerId, producerSequence, newSequence)) { - return null; - } - producerSequence = producerSequences.get(producerId); - } - } - catch (JsonProcessingException ex) { - throw new RuntimeException(ex); - } - catch (NumberFormatException ex) { - return Response - .status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of("error", "Producer sequence must be a number")) - .build(); - } - } - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java deleted file mode 100644 index 72a48b33a45a..000000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.segment.realtime.firehose; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.impl.InputRowParser; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; - -/** - * Firehose to give out only first n events from the delegate firehose. - */ -public class FixedCountFirehoseFactory implements FirehoseFactory -{ - private final FirehoseFactory delegate; - private final int count; - - @JsonCreator - public FixedCountFirehoseFactory( - @JsonProperty("delegate") FirehoseFactory delegate, - @JsonProperty("count") int count - ) - { - this.delegate = delegate; - this.count = count; - } - - @JsonProperty - public FirehoseFactory getDelegate() - { - return delegate; - } - - @JsonProperty - public int getCount() - { - return count; - } - - @Override - public Firehose connect(final InputRowParser parser, File temporaryDirectory) throws IOException - { - return new Firehose() - { - private int i = 0; - private final Firehose delegateFirehose = delegate.connect(parser, temporaryDirectory); - - @Override - public boolean hasMore() throws IOException - { - return i < count && delegateFirehose.hasMore(); - } - - @Nullable - @Override - public InputRow nextRow() throws IOException - { - Preconditions.checkArgument(i++ < count, "Max events limit reached."); - return delegateFirehose.nextRow(); - } - - @Override - public void close() throws IOException - { - delegateFirehose.close(); - } - }; - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehose.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehose.java deleted file mode 100644 index c0064a25f64c..000000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehose.java +++ /dev/null @@ -1,210 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.segment.realtime.firehose; - -import com.google.common.base.Function; -import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.java.util.common.DateTimes; -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.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.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.transform.TransformSpec; -import org.apache.druid.segment.transform.Transformer; - -import javax.annotation.Nullable; -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 IngestSegmentFirehose implements Firehose -{ - private final Transformer transformer; - private Yielder rowYielder; - - public IngestSegmentFirehose( - 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); - } - - @Override - public boolean hasMore() - { - return !rowYielder.isDone(); - } - - @Nullable - @Override - 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/server/src/main/java/org/apache/druid/segment/realtime/firehose/PredicateFirehose.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/PredicateFirehose.java deleted file mode 100644 index d6aadf07adc6..000000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/PredicateFirehose.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.segment.realtime.firehose; - -import com.google.common.base.Predicate; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.java.util.common.logger.Logger; - -import javax.annotation.Nullable; -import java.io.IOException; - -/** - * Provides a view on a firehose that only returns rows that match a certain predicate. - * Not thread-safe. - */ -public class PredicateFirehose implements Firehose -{ - private static final Logger log = new Logger(PredicateFirehose.class); - private static final int IGNORE_THRESHOLD = 5000; - private long ignored = 0; - - private final Firehose firehose; - private final Predicate predicate; - - @Nullable - private InputRow savedInputRow = null; - - public PredicateFirehose(Firehose firehose, Predicate predicate) - { - this.firehose = firehose; - this.predicate = predicate; - } - - @Override - public boolean hasMore() throws IOException - { - if (savedInputRow != null) { - return true; - } - - while (firehose.hasMore()) { - final InputRow row = firehose.nextRow(); - if (predicate.apply(row)) { - savedInputRow = row; - return true; - } - // Do not silently discard the rows - if (ignored % IGNORE_THRESHOLD == 0) { - log.warn("[%,d] InputRow(s) ignored as they do not satisfy the predicate", ignored); - } - ignored++; - } - - return false; - } - - @Nullable - @Override - public InputRow nextRow() - { - final InputRow row = savedInputRow; - savedInputRow = null; - return row; - } - - @Override - public void close() throws IOException - { - firehose.close(); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java deleted file mode 100644 index 9bfda42c8aac..000000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.segment.realtime.firehose; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.errorprone.annotations.concurrent.GuardedBy; -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.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.utils.CloseableUtils; -import org.joda.time.DateTime; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - -/** - * Creates firehoses that shut off at a particular time. Useful for limiting the lifespan of a realtime job. - * - * Each {@link Firehose} created by this factory spins up and manages one thread for calling {@link Firehose#close()} - * asynchronously at the specified {@link #shutoffTime}. - */ -@Deprecated -public class TimedShutoffFirehoseFactory implements FirehoseFactory -{ - private static final EmittingLogger log = new EmittingLogger(FirehoseFactory.class); - - private final FirehoseFactory delegateFactory; - private final DateTime shutoffTime; - - @JsonCreator - public TimedShutoffFirehoseFactory( - @JsonProperty("delegate") FirehoseFactory delegateFactory, - @JsonProperty("shutoffTime") DateTime shutoffTime - ) - { - this.delegateFactory = delegateFactory; - this.shutoffTime = shutoffTime; - } - - @Override - public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException - { - return new TimedShutoffFirehose(parser, temporaryDirectory); - } - - class TimedShutoffFirehose implements Firehose - { - private final Firehose firehose; - private final ScheduledExecutorService shutdownExec; - @GuardedBy("this") - private boolean closed = false; - - TimedShutoffFirehose(InputRowParser parser, File temporaryDirectory) throws IOException - { - firehose = delegateFactory.connect(parser, temporaryDirectory); - - shutdownExec = Execs.scheduledSingleThreaded("timed-shutoff-firehose-%d"); - - shutdownExec.schedule( - () -> { - log.info("Closing delegate firehose."); - - try { - TimedShutoffFirehose.this.close(); - } - catch (IOException e) { - log.warn(e, "Failed to close delegate firehose, ignoring."); - } - }, - shutoffTime.getMillis() - System.currentTimeMillis(), - TimeUnit.MILLISECONDS - ); - - log.info("Firehose created, will shut down at: %s", shutoffTime); - } - - @Override - public boolean hasMore() throws IOException - { - return firehose.hasMore(); - } - - @Nullable - @Override - public InputRow nextRow() throws IOException - { - return firehose.nextRow(); - } - - /** - * This method is synchronized because it might be called concurrently from multiple threads: from {@link - * #shutdownExec}, and explicitly on this Firehose object. - */ - @Override - public synchronized void close() throws IOException - { - if (!closed) { - closed = true; - CloseableUtils.closeAll(firehose, shutdownExec::shutdownNow); - } - } - } - - @JsonProperty("delegate") - public FirehoseFactory getDelegateFactory() - { - return delegateFactory; - } - - @JsonProperty("shutoffTime") - public DateTime getShutoffTime() - { - return shutoffTime; - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/package-info.java b/server/src/main/java/org/apache/druid/segment/realtime/package-info.java similarity index 94% rename from server/src/main/java/org/apache/druid/segment/realtime/firehose/package-info.java rename to server/src/main/java/org/apache/druid/segment/realtime/package-info.java index e4a06999f363..2f80f8a1a06c 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/package-info.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/package-info.java @@ -18,6 +18,6 @@ */ @EverythingIsNonnullByDefault -package org.apache.druid.segment.realtime.firehose; +package org.apache.druid.segment.realtime; import org.apache.druid.annotations.EverythingIsNonnullByDefault; diff --git a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java index cf407102c5a7..06c50af02986 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java +++ b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java @@ -79,7 +79,7 @@ public class BatchDataSegmentAnnouncer implements DataSegmentAnnouncer private final ChangeRequestHistory changes = new ChangeRequestHistory<>(); - private final ConcurrentMap taskSinkSchema = new ConcurrentHashMap<>(); + private final ConcurrentMap taskAppendableSegmentSchema = new ConcurrentHashMap<>(); @Nullable private final SegmentZNode dummyZnode; @@ -321,11 +321,11 @@ public void announceSegmentSchemas( SegmentSchemas segmentSchemasChange ) { - log.info("Announcing sink schema for task [%s], absolute schema [%s], delta schema [%s].", + log.info("Announcing appendable segment schema for task [%s], absolute schema [%s], delta schema [%s].", taskId, segmentSchemas, segmentSchemasChange ); - taskSinkSchema.put(taskId, segmentSchemas); + taskAppendableSegmentSchema.put(taskId, segmentSchemas); if (segmentSchemasChange != null) { changes.addChangeRequest(new SegmentSchemasChangeRequest(segmentSchemasChange)); @@ -336,7 +336,7 @@ public void announceSegmentSchemas( public void removeSegmentSchemasForTask(String taskId) { log.info("Unannouncing task [%s].", taskId); - taskSinkSchema.remove(taskId); + taskAppendableSegmentSchema.remove(taskId); } /** @@ -359,8 +359,8 @@ public SegmentChangeRequestLoad apply(DataSegment input) } ); - Iterable sinkSchema = Iterables.transform( - taskSinkSchema.values(), + Iterable appendableSegmentSchema = Iterables.transform( + taskAppendableSegmentSchema.values(), new Function() { @Override @@ -370,7 +370,7 @@ public SegmentSchemasChangeRequest apply(SegmentSchemas input) } } ); - Iterable changeRequestIterables = Iterables.concat(segments, sinkSchema); + Iterable changeRequestIterables = Iterables.concat(segments, appendableSegmentSchema); SettableFuture> future = SettableFuture.create(); future.set(ChangeRequestsSnapshot.success(changes.getLastCounter(), Lists.newArrayList(changeRequestIterables))); return future; diff --git a/server/src/main/java/org/apache/druid/server/coordination/DataSegmentAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/DataSegmentAnnouncer.java index 08368cbcabe3..21ca1434d20c 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/DataSegmentAnnouncer.java +++ b/server/src/main/java/org/apache/druid/server/coordination/DataSegmentAnnouncer.java @@ -39,8 +39,8 @@ public interface DataSegmentAnnouncer * Announces schema associated with all segments for the specified realtime task. * * @param taskId taskId - * @param segmentSchemas absolute schema for all sinks, in case the client requests full sync. - * @param segmentSchemasChange schema change for all sinks + * @param segmentSchemas absolute schema for all appendable segments, in case the client requests full sync. + * @param segmentSchemasChange schema change for all appendable segments */ void announceSegmentSchemas( String taskId, diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java index aead7b86c719..8ee3d5ba0814 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java @@ -31,7 +31,7 @@ import org.apache.druid.guice.annotations.RemoteChatHandler; import org.apache.druid.guice.annotations.Self; import org.apache.druid.java.util.common.lifecycle.Lifecycle; -import org.apache.druid.segment.realtime.firehose.ChatHandlerResource; +import org.apache.druid.segment.realtime.ChatHandlerResource; import org.apache.druid.server.DruidNode; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.initialization.TLSServerConfig; diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java index 687ca2ef5485..feb61965daa3 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java @@ -32,7 +32,7 @@ import org.apache.druid.guice.annotations.Self; import org.apache.druid.java.util.common.lifecycle.Lifecycle; import org.apache.druid.query.lookup.LookupModule; -import org.apache.druid.segment.realtime.firehose.ChatHandlerResource; +import org.apache.druid.segment.realtime.ChatHandlerResource; import org.apache.druid.server.DruidNode; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.initialization.TLSServerConfig; diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/TaskIdResponseHeaderFilterHolder.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/TaskIdResponseHeaderFilterHolder.java index 7dcd3b2c237a..c4c0cb8ce4b0 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/TaskIdResponseHeaderFilterHolder.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/TaskIdResponseHeaderFilterHolder.java @@ -21,7 +21,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.segment.realtime.firehose.ChatHandlerResource; +import org.apache.druid.segment.realtime.ChatHandlerResource; public class TaskIdResponseHeaderFilterHolder extends ResponseHeaderFilterHolder { diff --git a/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseMetric.java b/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseMetric.java deleted file mode 100644 index 13502e18493c..000000000000 --- a/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseMetric.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.server.metrics; - -/** - * An EventReceiverFirehoseMetric is an object with metrics about EventReceiverFirehose objects. - * It is not likely that anything other than an EventReceiverFirehose actually implements this. - * This interface is not part of the public API and backwards incompatible changes can occur without - * requiring a major (or even minor) version change. - * The interface's primary purpose is to be able to share metrics via the EventReceiverFirehoseRegister - * without exposing the entire EventReceiverFirehose - */ -public interface EventReceiverFirehoseMetric -{ - /** - * Return the current number of {@link org.apache.druid.data.input.InputRow} that are stored in the buffer. - */ - int getCurrentBufferSize(); - - /** - * Return the capacity of the buffer. - */ - int getCapacity(); - - /** - * Return the number of bytes received by the firehose. - */ - long getBytesReceived(); - - -} diff --git a/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseMonitor.java b/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseMonitor.java deleted file mode 100644 index 29c1808ad515..000000000000 --- a/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseMonitor.java +++ /dev/null @@ -1,90 +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.server.metrics; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.inject.Inject; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; -import org.apache.druid.java.util.metrics.AbstractMonitor; -import org.apache.druid.java.util.metrics.KeyedDiff; -import org.apache.druid.java.util.metrics.MonitorUtils; -import org.apache.druid.query.DruidMetrics; - -import java.util.Map; -import java.util.Properties; - -public class EventReceiverFirehoseMonitor extends AbstractMonitor -{ - - private final EventReceiverFirehoseRegister register; - private final KeyedDiff keyedDiff = new KeyedDiff(); - private final Map dimensions; - - @Inject - public EventReceiverFirehoseMonitor( - EventReceiverFirehoseRegister eventReceiverFirehoseRegister, - Properties props - ) - { - this.register = eventReceiverFirehoseRegister; - this.dimensions = MonitorsConfig.extractDimensions( - props, - Lists.newArrayList(DruidMetrics.DATASOURCE, DruidMetrics.TASK_ID, DruidMetrics.TASK_TYPE) - ); - } - - @Override - public boolean doMonitor(ServiceEmitter emitter) - { - for (Map.Entry entry : register.getMetrics()) { - final String serviceName = entry.getKey(); - final EventReceiverFirehoseMetric metric = entry.getValue(); - - final ServiceMetricEvent.Builder builder = createEventBuilder(serviceName) - .setDimension( - "bufferCapacity", - String.valueOf(metric.getCapacity()) - ); - emitter.emit(builder.setMetric("ingest/events/buffered", metric.getCurrentBufferSize())); - Map diff = keyedDiff.to( - serviceName, - ImmutableMap.of("ingest/bytes/received", metric.getBytesReceived()) - ); - if (diff != null) { - final ServiceMetricEvent.Builder eventBuilder = createEventBuilder(serviceName); - for (Map.Entry diffEntry : diff.entrySet()) { - emitter.emit(eventBuilder.setMetric(diffEntry.getKey(), diffEntry.getValue())); - } - } - } - - return true; - } - - private ServiceMetricEvent.Builder createEventBuilder(String serviceName) - { - ServiceMetricEvent.Builder builder = ServiceMetricEvent.builder() - .setDimension("serviceName", serviceName); - MonitorUtils.addDimensionsToBuilder(builder, dimensions); - return builder; - } -} diff --git a/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseRegister.java b/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseRegister.java deleted file mode 100644 index 66a022992e74..000000000000 --- a/server/src/main/java/org/apache/druid/server/metrics/EventReceiverFirehoseRegister.java +++ /dev/null @@ -1,56 +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.server.metrics; - -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.logger.Logger; - -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -public class EventReceiverFirehoseRegister -{ - - private static final Logger log = new Logger(EventReceiverFirehoseRegister.class); - - private final ConcurrentMap metrics = new ConcurrentHashMap<>(); - - public void register(String serviceName, EventReceiverFirehoseMetric metric) - { - log.info("Registering EventReceiverFirehoseMetric for service [%s]", serviceName); - if (metrics.putIfAbsent(serviceName, metric) != null) { - throw new ISE("Service [%s] is already registered!", serviceName); - } - } - - public Iterable> getMetrics() - { - return metrics.entrySet(); - } - - public void unregister(String serviceName) - { - log.info("Unregistering EventReceiverFirehoseMetric for service [%s]", serviceName); - if (metrics.remove(serviceName) == null) { - log.warn("Unregistering a non-exist service. Service [%s] never exists.", serviceName); - } - } -} diff --git a/server/src/main/java/org/apache/druid/server/metrics/MetricsModule.java b/server/src/main/java/org/apache/druid/server/metrics/MetricsModule.java index 278a170910ad..ab2cdb3811f0 100644 --- a/server/src/main/java/org/apache/druid/server/metrics/MetricsModule.java +++ b/server/src/main/java/org/apache/druid/server/metrics/MetricsModule.java @@ -91,7 +91,6 @@ public void configure(Binder binder) binder.bind(DataSourceTaskIdHolder.class).in(LazySingleton.class); - binder.bind(EventReceiverFirehoseRegister.class).in(LazySingleton.class); binder.bind(ExecutorServiceMonitor.class).in(LazySingleton.class); // Instantiate eagerly so that we get everything registered and put into the Lifecycle diff --git a/server/src/test/java/org/apache/druid/guice/FirehoseModuleTest.java b/server/src/test/java/org/apache/druid/guice/FirehoseModuleTest.java deleted file mode 100644 index 8ecc93dece2b..000000000000 --- a/server/src/test/java/org/apache/druid/guice/FirehoseModuleTest.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.guice; - -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.cfg.MapperConfig; -import com.fasterxml.jackson.databind.introspect.AnnotatedClass; -import com.fasterxml.jackson.databind.jsontype.NamedType; -import com.google.common.reflect.ClassPath; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.segment.realtime.firehose.ClippedFirehoseFactory; -import org.apache.druid.utils.JvmUtils; -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; -import java.lang.reflect.Modifier; -import java.net.URL; -import java.net.URLClassLoader; -import java.util.Collection; -import java.util.Set; -import java.util.function.Predicate; -import java.util.stream.Collectors; - -public class FirehoseModuleTest -{ - private static final Predicate IS_FIREHOSE_FACTORY = - c -> FirehoseFactory.class.isAssignableFrom(c) && !Modifier.isAbstract(c.getModifiers()); - - @Test - public void testAllFirehoseFactorySubtypesRegistered() throws IOException - { - ObjectMapper objectMapper = createObjectMapper(); - Set registeredSubtypeClasses = getFirehoseFactorySubtypeClasses(objectMapper); - String packageName = ClippedFirehoseFactory.class.getPackage().getName(); - Set expectedSubtypeClasses = getFirehoseFactoryClassesInPackage(packageName); - Assert.assertEquals(expectedSubtypeClasses, registeredSubtypeClasses); - } - - private static ObjectMapper createObjectMapper() - { - ObjectMapper objectMapper = new ObjectMapper(); - for (Module jacksonModule : new FirehoseModule().getJacksonModules()) { - objectMapper.registerModule(jacksonModule); - } - return objectMapper; - } - - private static Set getFirehoseFactorySubtypeClasses(ObjectMapper objectMapper) - { - Class parentClass = FirehoseFactory.class; - MapperConfig config = objectMapper.getDeserializationConfig(); - AnnotatedClass ac = AnnotatedClass.constructWithoutSuperTypes(parentClass, config); - Collection subtypes = objectMapper.getSubtypeResolver().collectAndResolveSubtypesByClass(config, ac); - Assert.assertNotNull(subtypes); - return subtypes.stream() - .map(NamedType::getType) - .filter(c -> !c.equals(parentClass)) - .collect(Collectors.toSet()); - } - - @SuppressWarnings("UnstableApiUsage") // for ClassPath - private static Set getFirehoseFactoryClassesInPackage(String packageName) throws IOException - { - // workaround for Guava 16, which can only parse the classpath from URLClassLoaders - // requires Guava 28 or later to work properly with the system class loader in Java 9 and above - URLClassLoader classloader = new URLClassLoader(JvmUtils.systemClassPath().toArray(new URL[0])); - ClassPath classPath = ClassPath.from(classloader); - return classPath.getTopLevelClasses(packageName).stream() - .map(ClassPath.ClassInfo::load) - .filter(IS_FIREHOSE_FACTORY) - .collect(Collectors.toSet()); - } -} - diff --git a/server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java b/server/src/test/java/org/apache/druid/segment/realtime/AppendableSegmentTest.java similarity index 77% rename from server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/AppendableSegmentTest.java index 9d85ec6c8e6b..fd3a3077643c 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/AppendableSegmentTest.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.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -45,7 +45,6 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.realtime.FireHydrant; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.NumberedShardSpec; @@ -68,7 +67,7 @@ /** * */ -public class SinkTest extends InitializedNullHandlingTest +public class AppendableSegmentTest extends InitializedNullHandlingTest { private static final ShardSpec SHARD_SPEC = new NumberedShardSpec(0, 1); private static final int MAX_ROWS_IN_MEMORY = 100; @@ -87,7 +86,7 @@ public void testSwap() throws Exception final Interval interval = Intervals.of("2013-01-01/2013-01-02"); final String version = DateTimes.nowUtc().toString(); - final Sink sink = new Sink( + final AppendableSegment appendableSegment = new AppendableSegment( interval, schema, SHARD_SPEC, @@ -98,7 +97,7 @@ public void testSwap() throws Exception true ); - sink.add( + appendableSegment.add( new InputRow() { @Override @@ -146,13 +145,13 @@ public int compareTo(Row o) false ); - FireHydrant currHydrant = sink.getCurrHydrant(); - Assert.assertEquals(Intervals.of("2013-01-01/PT1M"), currHydrant.getIndex().getInterval()); + PartialSegment currentPartialSegment = appendableSegment.getCurrentPartialSegment(); + Assert.assertEquals(Intervals.of("2013-01-01/PT1M"), currentPartialSegment.getIndex().getInterval()); - FireHydrant swapHydrant = sink.swap(); + PartialSegment swapPartialSegment = appendableSegment.swap(); - sink.add( + appendableSegment.add( new InputRow() { @Override @@ -200,11 +199,11 @@ public int compareTo(Row o) false ); - Assert.assertEquals(currHydrant, swapHydrant); - Assert.assertNotSame(currHydrant, sink.getCurrHydrant()); - Assert.assertEquals(Intervals.of("2013-01-01/PT1M"), sink.getCurrHydrant().getIndex().getInterval()); + Assert.assertEquals(currentPartialSegment, swapPartialSegment); + Assert.assertNotSame(currentPartialSegment, appendableSegment.getCurrentPartialSegment()); + Assert.assertEquals(Intervals.of("2013-01-01/PT1M"), appendableSegment.getCurrentPartialSegment().getIndex().getInterval()); - Assert.assertEquals(2, Iterators.size(sink.iterator())); + Assert.assertEquals(2, Iterators.size(appendableSegment.iterator())); } @Test @@ -212,20 +211,20 @@ public void testAcquireSegmentReferences_empty() { Assert.assertEquals( Collections.emptyList(), - Sink.acquireSegmentReferences(Collections.emptyList(), Function.identity(), false) + AppendableSegment.acquireSegmentReferences(Collections.emptyList(), Function.identity(), false) ); } @Test public void testAcquireSegmentReferences_two() throws IOException { - final List hydrants = twoHydrants(); - final List references = Sink.acquireSegmentReferences(hydrants, Function.identity(), false); + final List partialSegments = twoPartialSegments(); + final List references = AppendableSegment.acquireSegmentReferences(partialSegments, Function.identity(), false); Assert.assertNotNull(references); Assert.assertEquals(2, references.size()); - Assert.assertEquals(0, references.get(0).getHydrantNumber()); + Assert.assertEquals(0, references.get(0).getPartialSegmentNumber()); Assert.assertFalse(references.get(0).isImmutable()); - Assert.assertEquals(1, references.get(1).getHydrantNumber()); + Assert.assertEquals(1, references.get(1).getPartialSegmentNumber()); Assert.assertTrue(references.get(1).isImmutable()); CloseableUtils.closeAll(references); } @@ -233,11 +232,11 @@ public void testAcquireSegmentReferences_two() throws IOException @Test public void testAcquireSegmentReferences_two_skipIncremental() throws IOException { - final List hydrants = twoHydrants(); - final List references = Sink.acquireSegmentReferences(hydrants, Function.identity(), true); + final List partialSegments = twoPartialSegments(); + final List references = AppendableSegment.acquireSegmentReferences(partialSegments, Function.identity(), true); Assert.assertNotNull(references); Assert.assertEquals(1, references.size()); - Assert.assertEquals(1, references.get(0).getHydrantNumber()); + Assert.assertEquals(1, references.get(0).getPartialSegmentNumber()); Assert.assertTrue(references.get(0).isImmutable()); CloseableUtils.closeAll(references); } @@ -245,16 +244,16 @@ public void testAcquireSegmentReferences_two_skipIncremental() throws IOExceptio @Test public void testAcquireSegmentReferences_twoWithOneSwappedToNull() { - // One segment has been swapped out. (Happens when sinks are being closed.) - final List hydrants = twoHydrants(); - hydrants.get(1).swapSegment(null); + // One segment has been swapped out. (Happens when appendable segments are being closed.) + final List partialSegments = twoPartialSegments(); + partialSegments.get(1).swapSegment(null); - final List references = Sink.acquireSegmentReferences(hydrants, Function.identity(), false); + final List references = AppendableSegment.acquireSegmentReferences(partialSegments, Function.identity(), false); Assert.assertNull(references); } @Test - public void testGetSinkSignature() throws IndexSizeExceededException + public void testGetAppendableSegmentSignature() throws IndexSizeExceededException { final DataSchema schema = new DataSchema( "test", @@ -271,7 +270,7 @@ public void testGetSinkSignature() throws IndexSizeExceededException final Interval interval = Intervals.of("2013-01-01/2013-01-02"); final String version = DateTimes.nowUtc().toString(); - final Sink sink = new Sink( + final AppendableSegment appendableSegment = new AppendableSegment( interval, schema, SHARD_SPEC, @@ -282,7 +281,7 @@ public void testGetSinkSignature() throws IndexSizeExceededException true ); - sink.add(new MapBasedInputRow( + appendableSegment.add(new MapBasedInputRow( DateTimes.of("2013-01-01"), ImmutableList.of("dim1", "dimLong"), ImmutableMap.of("dim1", "value1", "dimLong", "20") @@ -294,10 +293,10 @@ public void testGetSinkSignature() throws IndexSizeExceededException expectedColumnTypeMap.put("dimLong", ColumnType.LONG); expectedColumnTypeMap.put("rows", ColumnType.LONG); - RowSignature signature = sink.getSignature(); + RowSignature signature = appendableSegment.getSignature(); Assert.assertEquals(toRowSignature(expectedColumnTypeMap), signature); - sink.add(new MapBasedInputRow( + appendableSegment.add(new MapBasedInputRow( DateTimes.of("2013-01-01"), ImmutableList.of("dim1", "dimLong", "newCol1"), ImmutableMap.of("dim1", "value2", "dimLong", "30", "newCol1", "value") @@ -306,40 +305,40 @@ public void testGetSinkSignature() throws IndexSizeExceededException expectedColumnTypeMap.remove("rows"); expectedColumnTypeMap.put("newCol1", ColumnType.STRING); expectedColumnTypeMap.put("rows", ColumnType.LONG); - signature = sink.getSignature(); + signature = appendableSegment.getSignature(); Assert.assertEquals(toRowSignature(expectedColumnTypeMap), signature); - sink.swap(); + appendableSegment.swap(); - sink.add(new MapBasedInputRow( + appendableSegment.add(new MapBasedInputRow( DateTimes.of("2013-01-01"), ImmutableList.of("dim1", "dimLong", "newCol2"), ImmutableMap.of("dim1", "value3", "dimLong", "30", "newCol2", "value") ), false); expectedColumnTypeMap.put("newCol2", ColumnType.STRING); - signature = sink.getSignature(); + signature = appendableSegment.getSignature(); Assert.assertEquals(toRowSignature(expectedColumnTypeMap), signature); - sink.add(new MapBasedInputRow( + appendableSegment.add(new MapBasedInputRow( DateTimes.of("2013-01-01"), ImmutableList.of("dim1", "dimLong", "newCol3"), ImmutableMap.of("dim1", "value3", "dimLong", "30", "newCol3", "value") ), false); expectedColumnTypeMap.put("newCol3", ColumnType.STRING); - signature = sink.getSignature(); + signature = appendableSegment.getSignature(); Assert.assertEquals(toRowSignature(expectedColumnTypeMap), signature); - sink.swap(); + appendableSegment.swap(); - sink.add(new MapBasedInputRow( + appendableSegment.add(new MapBasedInputRow( DateTimes.of("2013-01-01"), ImmutableList.of("dim1", "dimLong", "newCol4"), ImmutableMap.of("dim1", "value3", "dimLong", "30", "newCol4", "value") ), false); expectedColumnTypeMap.put("newCol4", ColumnType.STRING); - signature = sink.getSignature(); + signature = appendableSegment.getSignature(); Assert.assertEquals(toRowSignature(expectedColumnTypeMap), signature); } @@ -355,14 +354,14 @@ private RowSignature toRowSignature(Map columnTypeMap) } /** - * Generate one in-memory hydrant, one not-in-memory hydrant. + * Generate one in-memory partial segment, one not-in-memory partial segment. */ - private static List twoHydrants() + private static List twoPartialSegments() { final SegmentId segmentId = SegmentId.dummy("foo"); return Arrays.asList( - new FireHydrant(EasyMock.createMock(IncrementalIndex.class), 0, segmentId), - new FireHydrant( + new PartialSegment(EasyMock.createMock(IncrementalIndex.class), 0, segmentId), + new PartialSegment( new RowBasedSegment<>( segmentId, Sequences.empty(), diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResourceTest.java b/server/src/test/java/org/apache/druid/segment/realtime/ChatHandlerResourceTest.java similarity index 97% rename from server/src/test/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResourceTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/ChatHandlerResourceTest.java index 870636fb416a..21d99c622255 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResourceTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/ChatHandlerResourceTest.java @@ -18,7 +18,7 @@ */ -package org.apache.druid.segment.realtime.firehose; +package org.apache.druid.segment.realtime; import com.google.common.base.Optional; import org.apache.druid.server.initialization.jetty.ServiceUnavailableException; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/FireHydrantTest.java b/server/src/test/java/org/apache/druid/segment/realtime/PartialSegmentTest.java similarity index 76% rename from server/src/test/java/org/apache/druid/segment/realtime/FireHydrantTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/PartialSegmentTest.java index 38c3fda1e7e2..0bbe763b29c8 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/FireHydrantTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/PartialSegmentTest.java @@ -43,11 +43,11 @@ import java.util.Optional; import java.util.function.Function; -public class FireHydrantTest extends InitializedNullHandlingTest +public class PartialSegmentTest extends InitializedNullHandlingTest { private IncrementalIndexSegment incrementalIndexSegment; private QueryableIndexSegment queryableIndexSegment; - private FireHydrant hydrant; + private PartialSegment partialSegment; @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -58,15 +58,15 @@ public void setup() incrementalIndexSegment = new IncrementalIndexSegment(TestIndex.getIncrementalTestIndex(), SegmentId.dummy("test")); queryableIndexSegment = new QueryableIndexSegment(TestIndex.getMMappedTestIndex(), SegmentId.dummy("test")); - // hydrant starts out with incremental segment loaded - hydrant = new FireHydrant(incrementalIndexSegment, 0); + // partial segment starts out with incremental segment loaded + partialSegment = new PartialSegment(incrementalIndexSegment, 0); } @Test public void testGetIncrementedSegmentNotSwapped() { - Assert.assertEquals(0, hydrant.getHydrantSegment().getNumReferences()); - ReferenceCountingSegment segment = hydrant.getIncrementedSegment(); + Assert.assertEquals(0, partialSegment.getPartialSegment().getNumReferences()); + ReferenceCountingSegment segment = partialSegment.getIncrementedSegment(); Assert.assertNotNull(segment); Assert.assertTrue(segment.getBaseSegment() == incrementalIndexSegment); Assert.assertEquals(1, segment.getNumReferences()); @@ -75,10 +75,10 @@ public void testGetIncrementedSegmentNotSwapped() @Test public void testGetIncrementedSegmentSwapped() { - ReferenceCountingSegment incrementalSegmentReference = hydrant.getHydrantSegment(); + ReferenceCountingSegment incrementalSegmentReference = partialSegment.getPartialSegment(); Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); - hydrant.swapSegment(queryableIndexSegment); - ReferenceCountingSegment segment = hydrant.getIncrementedSegment(); + partialSegment.swapSegment(queryableIndexSegment); + ReferenceCountingSegment segment = partialSegment.getIncrementedSegment(); Assert.assertNotNull(segment); Assert.assertTrue(segment.getBaseSegment() == queryableIndexSegment); Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); @@ -89,19 +89,19 @@ public void testGetIncrementedSegmentSwapped() public void testGetIncrementedSegmentClosed() { expectedException.expect(ISE.class); - expectedException.expectMessage("segment.close() is called somewhere outside FireHydrant.swapSegment()"); - hydrant.getHydrantSegment().close(); - Assert.assertEquals(0, hydrant.getHydrantSegment().getNumReferences()); - ReferenceCountingSegment segment = hydrant.getIncrementedSegment(); + expectedException.expectMessage("segment.close() is called somewhere outside PartialSegment.swapSegment()"); + partialSegment.getPartialSegment().close(); + Assert.assertEquals(0, partialSegment.getPartialSegment().getNumReferences()); + ReferenceCountingSegment segment = partialSegment.getIncrementedSegment(); } @Test public void testGetAndIncrementSegment() throws IOException { - ReferenceCountingSegment incrementalSegmentReference = hydrant.getHydrantSegment(); + ReferenceCountingSegment incrementalSegmentReference = partialSegment.getPartialSegment(); Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); - Pair segmentAndCloseable = hydrant.getAndIncrementSegment(); + Pair segmentAndCloseable = partialSegment.getAndIncrementSegment(); Assert.assertEquals(1, segmentAndCloseable.lhs.getNumReferences()); segmentAndCloseable.rhs.close(); Assert.assertEquals(0, segmentAndCloseable.lhs.getNumReferences()); @@ -110,10 +110,10 @@ public void testGetAndIncrementSegment() throws IOException @Test public void testGetSegmentForQuery() throws IOException { - ReferenceCountingSegment incrementalSegmentReference = hydrant.getHydrantSegment(); + ReferenceCountingSegment incrementalSegmentReference = partialSegment.getPartialSegment(); Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); - Optional> maybeSegmentAndCloseable = hydrant.getSegmentForQuery( + Optional> maybeSegmentAndCloseable = partialSegment.getSegmentForQuery( Function.identity() ); Assert.assertTrue(maybeSegmentAndCloseable.isPresent()); @@ -127,13 +127,13 @@ public void testGetSegmentForQuery() throws IOException @Test public void testGetSegmentForQuerySwapped() throws IOException { - ReferenceCountingSegment incrementalSegmentReference = hydrant.getHydrantSegment(); - hydrant.swapSegment(queryableIndexSegment); - ReferenceCountingSegment queryableSegmentReference = hydrant.getHydrantSegment(); + ReferenceCountingSegment incrementalSegmentReference = partialSegment.getPartialSegment(); + partialSegment.swapSegment(queryableIndexSegment); + ReferenceCountingSegment queryableSegmentReference = partialSegment.getPartialSegment(); Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); Assert.assertEquals(0, queryableSegmentReference.getNumReferences()); - Optional> maybeSegmentAndCloseable = hydrant.getSegmentForQuery( + Optional> maybeSegmentAndCloseable = partialSegment.getSegmentForQuery( Function.identity() ); Assert.assertTrue(maybeSegmentAndCloseable.isPresent()); @@ -149,13 +149,13 @@ public void testGetSegmentForQuerySwapped() throws IOException @Test public void testGetSegmentForQuerySwappedWithNull() { - ReferenceCountingSegment incrementalSegmentReference = hydrant.getHydrantSegment(); - hydrant.swapSegment(null); - ReferenceCountingSegment queryableSegmentReference = hydrant.getHydrantSegment(); + ReferenceCountingSegment incrementalSegmentReference = partialSegment.getPartialSegment(); + partialSegment.swapSegment(null); + ReferenceCountingSegment queryableSegmentReference = partialSegment.getPartialSegment(); Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); Assert.assertNull(queryableSegmentReference); - Optional> maybeSegmentAndCloseable = hydrant.getSegmentForQuery( + Optional> maybeSegmentAndCloseable = partialSegment.getSegmentForQuery( Function.identity() ); Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); @@ -165,10 +165,10 @@ public void testGetSegmentForQuerySwappedWithNull() @Test public void testGetSegmentForQueryButNotAbleToAcquireReferences() { - ReferenceCountingSegment incrementalSegmentReference = hydrant.getHydrantSegment(); + ReferenceCountingSegment incrementalSegmentReference = partialSegment.getPartialSegment(); Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); - Optional> maybeSegmentAndCloseable = hydrant.getSegmentForQuery( + Optional> maybeSegmentAndCloseable = partialSegment.getSegmentForQuery( segmentReference -> new SegmentReference() { @Override @@ -217,12 +217,12 @@ public void close() public void testGetSegmentForQueryButNotAbleToAcquireReferencesSegmentClosed() { expectedException.expect(ISE.class); - expectedException.expectMessage("segment.close() is called somewhere outside FireHydrant.swapSegment()"); - ReferenceCountingSegment incrementalSegmentReference = hydrant.getHydrantSegment(); + expectedException.expectMessage("segment.close() is called somewhere outside PartialSegment.swapSegment()"); + ReferenceCountingSegment incrementalSegmentReference = partialSegment.getPartialSegment(); Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); incrementalSegmentReference.close(); - Optional> maybeSegmentAndCloseable = hydrant.getSegmentForQuery( + Optional> maybeSegmentAndCloseable = partialSegment.getSegmentForQuery( Function.identity() ); } @@ -231,7 +231,7 @@ public void testGetSegmentForQueryButNotAbleToAcquireReferencesSegmentClosed() @SuppressWarnings("ReturnValueIgnored") public void testToStringWhenSwappedWithNull() { - hydrant.swapSegment(null); - hydrant.toString(); + partialSegment.swapSegment(null); + partialSegment.toString(); } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProviderTest.java b/server/src/test/java/org/apache/druid/segment/realtime/ServiceAnnouncingChatHandlerProviderTest.java similarity index 99% rename from server/src/test/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProviderTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/ServiceAnnouncingChatHandlerProviderTest.java index 71a3fe308f4d..05fb11e4b620 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/ServiceAnnouncingChatHandlerProviderTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/ServiceAnnouncingChatHandlerProviderTest.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.curator.discovery.ServiceAnnouncer; import org.apache.druid.server.DruidNode; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SinkSchemaUtilTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppendableSegmentSchemaUtilTest.java similarity index 86% rename from server/src/test/java/org/apache/druid/segment/realtime/appenderator/SinkSchemaUtilTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppendableSegmentSchemaUtilTest.java index 2d0da3a33bc3..b03bf56a74c4 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SinkSchemaUtilTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppendableSegmentSchemaUtilTest.java @@ -36,18 +36,18 @@ import java.util.Optional; import java.util.stream.Collectors; -public class SinkSchemaUtilTest +public class AppendableSegmentSchemaUtilTest { @Test public void testComputeAbsoluteSchemaEmpty() { - Assert.assertEquals(Optional.empty(), SinkSchemaUtil.computeAbsoluteSchema(new HashMap<>())); + Assert.assertEquals(Optional.empty(), AppendableSegmentSchemaUtil.computeAbsoluteSchema(new HashMap<>())); } @Test public void testComputeAbsoluteSchema() { - Map> sinkSchemaMap = new HashMap<>(); + Map> appendableSegmentSchemaMap = new HashMap<>(); SegmentId segment1 = SegmentId.of( "foo", @@ -60,7 +60,7 @@ public void testComputeAbsoluteSchema() columnTypeMap1.put("dim2", ColumnType.UNKNOWN_COMPLEX); columnTypeMap1.put("dim3", ColumnType.NESTED_DATA); Pair schema1 = Pair.of(toRowSignature(columnTypeMap1), 20); - sinkSchemaMap.put(segment1, schema1); + appendableSegmentSchemaMap.put(segment1, schema1); SegmentId segment2 = SegmentId.of( "foo", @@ -75,9 +75,9 @@ public void testComputeAbsoluteSchema() columnTypeMap2.put("dim3", ColumnType.STRING); columnTypeMap2.put("dim4", ColumnType.NESTED_DATA); Pair schema2 = Pair.of(toRowSignature(columnTypeMap2), 40); - sinkSchemaMap.put(segment2, schema2); + appendableSegmentSchemaMap.put(segment2, schema2); - Optional segmentSchemas = SinkSchemaUtil.computeAbsoluteSchema(sinkSchemaMap); + Optional segmentSchemas = AppendableSegmentSchemaUtil.computeAbsoluteSchema(appendableSegmentSchemaMap); Assert.assertTrue(segmentSchemas.isPresent()); Assert.assertEquals(2, segmentSchemas.get().getSegmentSchemaList().size()); @@ -108,7 +108,7 @@ public void testComputeAbsoluteSchema() @Test public void testComputeSchemaChangeNoChange() { - Map> previousSinkSchemaMap = new HashMap<>(); + Map> previousAppendableSegmentSchemaMap = new HashMap<>(); SegmentId segment1 = SegmentId.of( "foo", @@ -121,7 +121,7 @@ public void testComputeSchemaChangeNoChange() columnTypeMap1.put("dim2", ColumnType.UNKNOWN_COMPLEX); columnTypeMap1.put("dim3", ColumnType.NESTED_DATA); Pair schema1 = Pair.of(toRowSignature(columnTypeMap1), 20); - previousSinkSchemaMap.put(segment1, schema1); + previousAppendableSegmentSchemaMap.put(segment1, schema1); SegmentId segment2 = SegmentId.of( "foo", @@ -136,7 +136,7 @@ public void testComputeSchemaChangeNoChange() columnTypeMap2.put("dim3", ColumnType.STRING); columnTypeMap2.put("dim4", ColumnType.NESTED_DATA); Pair schema2 = Pair.of(toRowSignature(columnTypeMap2), 40); - previousSinkSchemaMap.put(segment2, schema2); + previousAppendableSegmentSchemaMap.put(segment2, schema2); SegmentId segment3 = SegmentId.of( "foo", @@ -151,15 +151,15 @@ public void testComputeSchemaChangeNoChange() columnTypeMap2.put("dim3", ColumnType.STRING); columnTypeMap2.put("dim5", ColumnType.NESTED_DATA); Pair schema3 = Pair.of(toRowSignature(columnTypeMap3), 80); - previousSinkSchemaMap.put(segment3, schema3); + previousAppendableSegmentSchemaMap.put(segment3, schema3); - Assert.assertFalse(SinkSchemaUtil.computeSchemaChange(previousSinkSchemaMap, previousSinkSchemaMap).isPresent()); + Assert.assertFalse(AppendableSegmentSchemaUtil.computeSchemaChange(previousAppendableSegmentSchemaMap, previousAppendableSegmentSchemaMap).isPresent()); } @Test public void testComputeSchemaChange() { - Map> previousSinkSchemaMap = new HashMap<>(); + Map> previousAppendableSegmentSchemaMap = new HashMap<>(); SegmentId segment1 = SegmentId.of( "foo", @@ -172,7 +172,7 @@ public void testComputeSchemaChange() columnTypeMap1.put("dim2", ColumnType.UNKNOWN_COMPLEX); columnTypeMap1.put("dim3", ColumnType.NESTED_DATA); Pair schema1 = Pair.of(toRowSignature(columnTypeMap1), 20); - previousSinkSchemaMap.put(segment1, schema1); + previousAppendableSegmentSchemaMap.put(segment1, schema1); SegmentId segment2 = SegmentId.of( "foo", @@ -187,7 +187,7 @@ public void testComputeSchemaChange() columnTypeMap2.put("dim3", ColumnType.STRING); columnTypeMap2.put("dim4", ColumnType.NESTED_DATA); Pair schema2 = Pair.of(toRowSignature(columnTypeMap2), 40); - previousSinkSchemaMap.put(segment2, schema2); + previousAppendableSegmentSchemaMap.put(segment2, schema2); SegmentId segment3 = SegmentId.of( "foo", @@ -202,9 +202,9 @@ public void testComputeSchemaChange() columnTypeMap2.put("dim3", ColumnType.STRING); columnTypeMap2.put("dim5", ColumnType.NESTED_DATA); Pair schema3 = Pair.of(toRowSignature(columnTypeMap3), 80); - previousSinkSchemaMap.put(segment3, schema3); + previousAppendableSegmentSchemaMap.put(segment3, schema3); - Map> currentSinkSchemaMap = new HashMap<>(); + Map> currentAppendableSegmentSchemaMap = new HashMap<>(); // new columns and numRows changed for segment1 Map currColumnTypeMap1 = Maps.newLinkedHashMap(); @@ -213,14 +213,14 @@ public void testComputeSchemaChange() currColumnTypeMap1.put("dim4", ColumnType.NESTED_DATA); currColumnTypeMap1.put("dim5", ColumnType.STRING); Pair currSchema1 = Pair.of(toRowSignature(currColumnTypeMap1), 50); - currentSinkSchemaMap.put(segment1, currSchema1); + currentAppendableSegmentSchemaMap.put(segment1, currSchema1); // no change for segment2 - currentSinkSchemaMap.put(segment2, schema2); + currentAppendableSegmentSchemaMap.put(segment2, schema2); // numRows changes for segment3 Pair currSchema3 = Pair.of(toRowSignature(columnTypeMap3), 100); - currentSinkSchemaMap.put(segment3, currSchema3); + currentAppendableSegmentSchemaMap.put(segment3, currSchema3); SegmentId segment4 = SegmentId.of( "foo", @@ -235,9 +235,9 @@ public void testComputeSchemaChange() columnTypeMap4.put("dim3", ColumnType.STRING); columnTypeMap4.put("dim4", ColumnType.NESTED_DATA); Pair schema4 = Pair.of(toRowSignature(columnTypeMap4), 40); - currentSinkSchemaMap.put(segment4, schema4); + currentAppendableSegmentSchemaMap.put(segment4, schema4); - Optional segmentSchemasChange = SinkSchemaUtil.computeSchemaChange(previousSinkSchemaMap, currentSinkSchemaMap); + Optional segmentSchemasChange = AppendableSegmentSchemaUtil.computeSchemaChange(previousAppendableSegmentSchemaMap, currentAppendableSegmentSchemaMap); Assert.assertTrue(segmentSchemasChange.isPresent()); Assert.assertEquals(3, segmentSchemasChange.get().getSegmentSchemaList().size()); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsTest.java index 66b2281d33cc..2a4ce6f24f7e 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsTest.java @@ -80,7 +80,7 @@ public void testClosedSegmentsOfflineAppenderator() throws Exception } @Test - public void testClosedSegmentsSinksOfflineAppenderator() throws Exception + public void testClosedAppendableSegmentsOfflineAppenderator() throws Exception { try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS_SINKS")) { Assert.assertTrue(tester.appenderator instanceof BatchAppenderator); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsAppendableSegmentsBatchAppenderatorTest.java similarity index 83% rename from server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorTest.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsAppendableSegmentsBatchAppenderatorTest.java index 55b5f2355067..02291eb4eb59 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsAppendableSegmentsBatchAppenderatorTest.java @@ -49,7 +49,7 @@ import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; -public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHandlingTest +public class ClosedSegmentsAppendableSegmentsBatchAppenderatorTest extends InitializedNullHandlingTest { private static final List IDENTIFIERS = ImmutableList.of( createSegmentId("2000/2001", "A", 0), // should be in seg_0 @@ -60,14 +60,14 @@ public class ClosedSegmentsSinksBatchAppenderatorTest extends InitializedNullHan @Test public void testSimpleIngestion() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, false)) { + try (final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(3, false)) { final Appenderator appenderator = tester.getAppenderator(); // startJob Assert.assertNull(appenderator.startJob()); // getDataSource - Assert.assertEquals(ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); + Assert.assertEquals(ClosedSegmentsAppendableSegmentsBatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); // add #1 Assert.assertEquals( @@ -96,7 +96,7 @@ public void testSimpleIngestion() throws Exception .getNumRowsInSegment() ); - // since we just added three rows and the max rows in memory is three, all the segments (sinks etc.) + // since we just added three rows and the max rows in memory is three, all the segments (appendable seg etc.) // above should be cleared now Assert.assertEquals( Collections.emptyList(), @@ -136,14 +136,14 @@ public void testSimpleIngestion() throws Exception @Test public void testPushFailure() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, true)) { + try (final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(3, true)) { final Appenderator appenderator = tester.getAppenderator(); // startJob Assert.assertNull(appenderator.startJob()); // getDataSource - Assert.assertEquals(ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); + Assert.assertEquals(ClosedSegmentsAppendableSegmentsBatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); // add #1 Assert.assertEquals( @@ -172,7 +172,7 @@ public void testPushFailure() throws Exception .getNumRowsInSegment() ); - // since we just added three rows and the max rows in memory is three, all the segments (sinks etc.) + // since we just added three rows and the max rows in memory is three, all the segments (appendable segs etc.) // above should be cleared now Assert.assertEquals( Collections.emptyList(), @@ -219,14 +219,14 @@ public void testPushFailure() throws Exception @Test public void testPeriodGranularityNonUTCIngestion() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(1, false)) { + try (final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(1, false)) { final Appenderator appenderator = tester.getAppenderator(); // startJob Assert.assertNull(appenderator.startJob()); // getDataSource - Assert.assertEquals(ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); + Assert.assertEquals(ClosedSegmentsAppendableSegmentsBatchAppenderatorTester.DATASOURCE, appenderator.getDataSource()); // Create a segment identifier with a non-utc interval SegmentIdWithShardSpec segmentIdWithNonUTCTime = @@ -246,7 +246,7 @@ public void testPeriodGranularityNonUTCIngestion() throws Exception ); - // since we just added one row and the max rows in memory is one, all the segments (sinks etc) + // since we just added one row and the max rows in memory is one, all the segments (appendable segs etc) // above should be cleared now Assert.assertEquals( Collections.emptyList(), @@ -281,7 +281,7 @@ public void testPeriodGranularityNonUTCIngestion() throws Exception public void testMaxBytesInMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester( + final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester( 100, 1024, null, @@ -313,10 +313,10 @@ public void testMaxBytesInMemoryWithSkipBytesInMemoryOverheadCheckConfig() throw } @Test - public void testMaxBytesInMemoryInMultipleSinksWithSkipBytesInMemoryOverheadCheckConfig() throws Exception + public void testMaxBytesInMemoryInMultipleAppendableSegmentsWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester( + final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester( 100, 1024, null, @@ -346,7 +346,7 @@ public void testMaxBytesInMemoryInMultipleSinksWithSkipBytesInMemoryOverheadChec @Test public void testMaxBytesInMemory() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(100, 15000, true)) { + try (final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(100, 15000, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -355,31 +355,31 @@ public void testMaxBytesInMemory() throws Exception //expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 54 (dimsKeySize) = 182 + 1 byte when null handling is enabled int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; int currentInMemoryIndexSize = 182 + nullHandlingOverhead; - int sinkSizeOverhead = BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; - // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet + int appendableSegmentSizeOverhead = BatchAppenderator.ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT; + // currPartialSegment in the appendable segment still has > 0 bytesInMemory since we do not persist yet Assert.assertEquals( currentInMemoryIndexSize, ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( - currentInMemoryIndexSize + sinkSizeOverhead, + currentInMemoryIndexSize + appendableSegmentSizeOverhead, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() ); - // We do multiple more adds to the same sink to cause persist. + // We do multiple more adds to the same appendable segment to cause persist. for (int i = 0; i < 53; i++) { appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar_" + i, 1), null); } - // currHydrant size is 0 since we just persist all indexes to disk. + // currPartialSegment size is 0 since we just persist all indexes to disk. currentInMemoryIndexSize = 0; // We are now over maxSizeInBytes after the add. Hence, we do a persist. - // currHydrant in the sink has 0 bytesInMemory since we just did a persist + // currPartialSegment in the appendable segment has 0 bytesInMemory since we just did a persist Assert.assertEquals( currentInMemoryIndexSize, ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); - // no sinks no hydrants after a persist, so we should have zero bytes currently in memory + // no appendable segments no partial segments after a persist, so we should have zero bytes currently in memory Assert.assertEquals( currentInMemoryIndexSize, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() @@ -387,26 +387,26 @@ public void testMaxBytesInMemory() throws Exception // Add a single row after persisted appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bob", 1), null); - // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet + // currPartialSegment in the appendable segment still has > 0 bytesInMemory since we do not persist yet currentInMemoryIndexSize = 182 + nullHandlingOverhead; Assert.assertEquals( currentInMemoryIndexSize, ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( - currentInMemoryIndexSize + sinkSizeOverhead, + currentInMemoryIndexSize + appendableSegmentSizeOverhead, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() ); - // We do multiple more adds to the same sink to cause persist. + // We do multiple more adds to the same appendable segment to cause persist. for (int i = 0; i < 53; i++) { appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar_" + i, 1), null); } - // currHydrant size is 0 since we just persist all indexes to disk. + // currPartialSegment size is 0 since we just persist all indexes to disk. currentInMemoryIndexSize = 0; // We are now over maxSizeInBytes after the add. Hence, we do a persist. - // so no sinks & hydrants should be in memory... + // so no appendable segments & partial segments should be in memory... Assert.assertEquals( currentInMemoryIndexSize, ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) @@ -425,8 +425,8 @@ public void testMaxBytesInMemory() throws Exception @Test(expected = RuntimeException.class, timeout = 5000L) public void testTaskFailAsPersistCannotFreeAnyMoreMemory() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(100, 5180, true)) { + try (final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = + new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(100, 5180, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo", 1), null); @@ -437,7 +437,7 @@ public void testTaskFailAsPersistCannotFreeAnyMoreMemory() throws Exception public void testTaskDoesNotFailAsExceededMemoryWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester( + final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester( 100, 10, null, @@ -468,8 +468,8 @@ public void testTaskDoesNotFailAsExceededMemoryWithSkipBytesInMemoryOverheadChec @Test public void testTaskCleanupInMemoryCounterAfterCloseWithRowInMemory() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(100, 10000, true)) { + try (final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = + new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(100, 10000, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -478,9 +478,9 @@ public void testTaskCleanupInMemoryCounterAfterCloseWithRowInMemory() throws Exc // Still under maxSizeInBytes after the add. Hence, we do not persist yet int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; int currentInMemoryIndexSize = 182 + nullHandlingOverhead; - int sinkSizeOverhead = BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; + int appendableSegmentSizeOverhead = BatchAppenderator.ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT; Assert.assertEquals( - currentInMemoryIndexSize + sinkSizeOverhead, + currentInMemoryIndexSize + appendableSegmentSizeOverhead, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() ); @@ -493,10 +493,10 @@ public void testTaskCleanupInMemoryCounterAfterCloseWithRowInMemory() throws Exc } @Test - public void testMaxBytesInMemoryInMultipleSinks() throws Exception + public void testMaxBytesInMemoryInMultipleAppendableSegments() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(1000, 28748, true)) { + try (final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = + new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(1000, 28748, true)) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -508,8 +508,8 @@ public void testMaxBytesInMemoryInMultipleSinks() throws Exception //expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 54 (dimsKeySize) = 182 + 1 byte when null handling is enabled int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; int currentInMemoryIndexSize = 182 + nullHandlingOverhead; - int sinkSizeOverhead = 2 * BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; - // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet + int appendableSegmentSizeOverhead = 2 * BatchAppenderator.ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT; + // currPartialSegment in the appendable segment still has > 0 bytesInMemory since we do not persist yet Assert.assertEquals( currentInMemoryIndexSize, ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) @@ -519,21 +519,21 @@ public void testMaxBytesInMemoryInMultipleSinks() throws Exception ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); Assert.assertEquals( - (2 * currentInMemoryIndexSize) + sinkSizeOverhead, + (2 * currentInMemoryIndexSize) + appendableSegmentSizeOverhead, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() ); - // We do multiple more adds to the same sink to cause persist. + // We do multiple more adds to the same appendable segment to cause persist. for (int i = 0; i < 49; i++) { // these records are 186 bytes appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar_" + i, 1), null); appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar_" + i, 1), null); } - // sinks + currHydrant size is 0 since we just persist all indexes to disk. + // appendable segments + currPartialSegment size is 0 since we just persist all indexes to disk. currentInMemoryIndexSize = 0; // We are now over maxSizeInBytes after the add. Hence, we do a persist. - // currHydrant and the sink has 0 bytesInMemory since we just did a persist + // currPartialSegment and the appendable segment has 0 bytesInMemory since we just did a persist Assert.assertEquals( currentInMemoryIndexSize, ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) @@ -547,9 +547,9 @@ public void testMaxBytesInMemoryInMultipleSinks() throws Exception ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() ); - // Add a single row after persisted to sink 0 + // Add a single row after persisted to appendable segment 0 appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bob", 1), null); - // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet + // currPartialSegment in the appendable segment still has > 0 bytesInMemory since we do not persist yet currentInMemoryIndexSize = 182 + nullHandlingOverhead; Assert.assertEquals( currentInMemoryIndexSize, @@ -559,13 +559,13 @@ public void testMaxBytesInMemoryInMultipleSinks() throws Exception 0, ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); - // only one sink so far: - sinkSizeOverhead = BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; + // only one appendable segment so far: + appendableSegmentSizeOverhead = BatchAppenderator.ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT; Assert.assertEquals( - currentInMemoryIndexSize + sinkSizeOverhead, + currentInMemoryIndexSize + appendableSegmentSizeOverhead, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() ); - // Now add a single row to sink 1 + // Now add a single row to appendable segment 1 appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bob", 1), null); Assert.assertEquals( currentInMemoryIndexSize, @@ -575,23 +575,23 @@ public void testMaxBytesInMemoryInMultipleSinks() throws Exception currentInMemoryIndexSize, ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); - sinkSizeOverhead += BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; + appendableSegmentSizeOverhead += BatchAppenderator.ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT; Assert.assertEquals( - (2 * currentInMemoryIndexSize) + sinkSizeOverhead, + (2 * currentInMemoryIndexSize) + appendableSegmentSizeOverhead, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() ); - // We do multiple more adds to the both sink to cause persist. + // We do multiple more adds to the both appendable segment to cause persist. for (int i = 0; i < 49; i++) { // 186 bytes appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "bar_" + i, 1), null); appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bar_" + i, 1), null); } - // currHydrant size is 0 since we just persist all indexes to disk. + // currPartialSegment size is 0 since we just persist all indexes to disk. currentInMemoryIndexSize = 0; // We are now over maxSizeInBytes after the add. Hence, we do a persist. - // currHydrant in the sink has 0 bytesInMemory since we just did a persist + // currPartialSegment in the appendable segment has 0 bytesInMemory since we just did a persist Assert.assertEquals( currentInMemoryIndexSize, ((BatchAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) @@ -616,8 +616,8 @@ public void testMaxBytesInMemoryInMultipleSinks() throws Exception @Test public void testIgnoreMaxBytesInMemory() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(100, -1, true)) { + try (final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = + new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(100, -1, true)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -635,9 +635,9 @@ public void testIgnoreMaxBytesInMemory() throws Exception // we added two rows only, and we told that maxSizeInBytes should be ignored, so it should not have been // persisted: - int sinkSizeOverhead = 2 * BatchAppenderator.ROUGH_OVERHEAD_PER_SINK; + int appendableSegmentSizeOverhead = 2 * BatchAppenderator.ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT; Assert.assertEquals( - (364 + 2 * nullHandlingOverhead) + sinkSizeOverhead, + (364 + 2 * nullHandlingOverhead) + appendableSegmentSizeOverhead, ((BatchAppenderator) appenderator).getBytesCurrentlyInMemory() ); Assert.assertEquals(2, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -649,7 +649,7 @@ public void testIgnoreMaxBytesInMemory() throws Exception @Test public void testMaxRowsInMemory() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, false)) { + try (final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(3, false)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -680,9 +680,9 @@ public void testMaxRowsInMemory() throws Exception } @Test - public void testAllHydrantsAreRecovered() throws Exception + public void testAllPartialSegmentsAreRecovered() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(1, false)) { + try (final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(1, false)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -692,7 +692,7 @@ public void testAllHydrantsAreRecovered() throws Exception appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo2", 1), null); appenderator.add(IDENTIFIERS.get(0), createInputRow("2000", "foo3", 1), null); - // Since maxRowsInMemory is one there ought to be three hydrants stored and recovered + // Since maxRowsInMemory is one there ought to be three partial segments stored and recovered // just before push, internally the code has a sanity check to make sure that this works. If it does not it throws // an exception final SegmentsAndCommitMetadata segmentsAndCommitMetadata = appenderator.push( @@ -715,7 +715,7 @@ public void testAllHydrantsAreRecovered() throws Exception @Test public void testTotalRowsPerSegment() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, false)) { + try (final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(3, false)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -743,7 +743,7 @@ public void testTotalRowsPerSegment() throws Exception Assert.assertEquals(2, addResult1.getNumRowsInSegment()); // persist expected ^ (3) rows added - // total rows per segment ought to be preserved even when sinks are removed from memory: + // total rows per segment ought to be preserved even when appendable segments are removed from memory: addResult1 = appenderator.add(IDENTIFIERS.get(1), createInputRow("2000", "bat", 1), null); Assert.assertEquals(1, ((BatchAppenderator) appenderator).getRowsInMemory()); @@ -775,7 +775,7 @@ public void testTotalRowsPerSegment() throws Exception @Test public void testRestoreFromDisk() throws Exception { - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(2, false); + final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(2, false); final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -814,7 +814,7 @@ public void testRestoreFromDisk() throws Exception @Test public void testCleanupFromDiskAfterClose() throws Exception { - final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(2, false); + final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(2, false); final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -854,7 +854,7 @@ public void testCleanupFromDiskAfterClose() throws Exception @Test(timeout = 5000L) public void testTotalRowCount() throws Exception { - try (final ClosedSegmensSinksBatchAppenderatorTester tester = new ClosedSegmensSinksBatchAppenderatorTester(3, false)) { + try (final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(3, false)) { final Appenderator appenderator = tester.getAppenderator(); Assert.assertEquals(0, appenderator.getTotalRowCount()); @@ -895,10 +895,10 @@ public void testTotalRowCount() throws Exception public void testVerifyRowIngestionMetrics() throws Exception { final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(5, - 10000L, - null, false, rowIngestionMeters + try (final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = + new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(5, + 10000L, + null, false, rowIngestionMeters )) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -918,10 +918,10 @@ public void testVerifyRowIngestionMetrics() throws Exception public void testPushContract() throws Exception { final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(1, - 50000L, - null, false, rowIngestionMeters + try (final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = + new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(1, + 50000L, + null, false, rowIngestionMeters )) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -967,10 +967,10 @@ public void testPushContract() throws Exception public void testCloseContract() throws Exception { final RowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - try (final ClosedSegmensSinksBatchAppenderatorTester tester = - new ClosedSegmensSinksBatchAppenderatorTester(1, - 50000L, - null, false, rowIngestionMeters + try (final ClosedSegmentsAppendableSegmentsBatchAppenderatorTester tester = + new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(1, + 50000L, + null, false, rowIngestionMeters )) { final Appenderator appenderator = tester.getAppenderator(); appenderator.startJob(); @@ -1018,7 +1018,7 @@ public void testCloseContract() throws Exception private static SegmentIdWithShardSpec createNonUTCSegmentId(String interval, String version, int partitionNum) { return new SegmentIdWithShardSpec( - ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, + ClosedSegmentsAppendableSegmentsBatchAppenderatorTester.DATASOURCE, new Interval(interval, ISOChronology.getInstance(DateTimes.inferTzFromString("Asia/Seoul"))), version, new LinearShardSpec(partitionNum) @@ -1029,7 +1029,7 @@ private static SegmentIdWithShardSpec createNonUTCSegmentId(String interval, Str private static SegmentIdWithShardSpec createSegmentId(String interval, String version, int partitionNum) { return new SegmentIdWithShardSpec( - ClosedSegmensSinksBatchAppenderatorTester.DATASOURCE, + ClosedSegmentsAppendableSegmentsBatchAppenderatorTester.DATASOURCE, Intervals.of(interval), version, new LinearShardSpec(partitionNum) diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsAppendableSegmentsBatchAppenderatorTester.java similarity index 94% rename from server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java rename to server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsAppendableSegmentsBatchAppenderatorTester.java index cf2d7f798986..f0b293479367 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsAppendableSegmentsBatchAppenderatorTester.java @@ -59,7 +59,7 @@ import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; -public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable +public class ClosedSegmentsAppendableSegmentsBatchAppenderatorTester implements AutoCloseable { public static final String DATASOURCE = "foo"; @@ -72,14 +72,14 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable private final List pushedSegments = new CopyOnWriteArrayList<>(); - public ClosedSegmensSinksBatchAppenderatorTester( + public ClosedSegmentsAppendableSegmentsBatchAppenderatorTester( final int maxRowsInMemory ) { this(maxRowsInMemory, -1, null, false); } - public ClosedSegmensSinksBatchAppenderatorTester( + public ClosedSegmentsAppendableSegmentsBatchAppenderatorTester( final int maxRowsInMemory, final boolean enablePushFailure ) @@ -87,7 +87,7 @@ public ClosedSegmensSinksBatchAppenderatorTester( this(maxRowsInMemory, -1, null, enablePushFailure); } - public ClosedSegmensSinksBatchAppenderatorTester( + public ClosedSegmentsAppendableSegmentsBatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, final boolean enablePushFailure @@ -96,7 +96,7 @@ public ClosedSegmensSinksBatchAppenderatorTester( this(maxRowsInMemory, maxSizeInBytes, null, enablePushFailure); } - public ClosedSegmensSinksBatchAppenderatorTester( + public ClosedSegmentsAppendableSegmentsBatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, final File basePersistDirectory, @@ -113,7 +113,7 @@ public ClosedSegmensSinksBatchAppenderatorTester( ); } - public ClosedSegmensSinksBatchAppenderatorTester( + public ClosedSegmentsAppendableSegmentsBatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, @Nullable final File basePersistDirectory, @@ -126,7 +126,7 @@ public ClosedSegmensSinksBatchAppenderatorTester( ); } - public ClosedSegmensSinksBatchAppenderatorTester( + public ClosedSegmentsAppendableSegmentsBatchAppenderatorTester( final int maxRowsInMemory, final long maxSizeInBytes, @Nullable final File basePersistDirectory, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorDriverTest.java index cc5a7f282ebf..4a503368221b 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorDriverTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmentsSinksBatchAppenderatorDriverTest.java @@ -78,7 +78,7 @@ public class ClosedSegmentsSinksBatchAppenderatorDriverTest extends EasyMockSupp ); private SegmentAllocator allocator; - private ClosedSegmensSinksBatchAppenderatorTester appenderatorTester; + private ClosedSegmentsAppendableSegmentsBatchAppenderatorTester appenderatorTester; private BatchAppenderatorDriver driver; private DataSegmentKiller dataSegmentKiller; @@ -89,7 +89,7 @@ public class ClosedSegmentsSinksBatchAppenderatorDriverTest extends EasyMockSupp @Before public void setup() { - appenderatorTester = new ClosedSegmensSinksBatchAppenderatorTester(MAX_ROWS_IN_MEMORY); + appenderatorTester = new ClosedSegmentsAppendableSegmentsBatchAppenderatorTester(MAX_ROWS_IN_MEMORY); allocator = new TestSegmentAllocator(DATA_SOURCE, Granularities.HOUR); dataSegmentKiller = createStrictMock(DataSegmentKiller.class); driver = new BatchAppenderatorDriver( diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/CommittedTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/CommittedTest.java index d95a74dcfbff..d6cf2d40bb72 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/CommittedTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/CommittedTest.java @@ -61,10 +61,10 @@ public class CommittedTest private static Committed fixedInstance() { - final Map hydrants = new HashMap<>(); - hydrants.put(IDENTIFIER1, 3); - hydrants.put(IDENTIFIER2, 2); - return new Committed(hydrants, ImmutableMap.of("metadata", "foo")); + final Map partialSegments = new HashMap<>(); + partialSegments.put(IDENTIFIER1, 3); + partialSegments.put(IDENTIFIER2, 2); + return new Committed(partialSegments, ImmutableMap.of("metadata", "foo")); } @Test @@ -89,21 +89,21 @@ public void testSerde() throws Exception final Committed committed2 = OBJECT_MAPPER.readValue(bytes, Committed.class); Assert.assertEquals("Round trip: overall", committed, committed2); Assert.assertEquals("Round trip: metadata", committed.getMetadata(), committed2.getMetadata()); - Assert.assertEquals("Round trip: identifiers", committed.getHydrants().keySet(), committed2.getHydrants().keySet()); + Assert.assertEquals("Round trip: identifiers", committed.getPartialSegments().keySet(), committed2.getPartialSegments().keySet()); } @Test - public void testGetCommittedHydrant() + public void testGetCommittedPartialSegment() { - Assert.assertEquals(3, fixedInstance().getCommittedHydrants(IDENTIFIER1)); - Assert.assertEquals(2, fixedInstance().getCommittedHydrants(IDENTIFIER2)); - Assert.assertEquals(0, fixedInstance().getCommittedHydrants(IDENTIFIER3)); + Assert.assertEquals(3, fixedInstance().getCommittedPartialSegments(IDENTIFIER1)); + Assert.assertEquals(2, fixedInstance().getCommittedPartialSegments(IDENTIFIER2)); + Assert.assertEquals(0, fixedInstance().getCommittedPartialSegments(IDENTIFIER3)); } @Test public void testWithout() { - Assert.assertEquals(0, fixedInstance().without(IDENTIFIER1).getCommittedHydrants(IDENTIFIER1)); - Assert.assertEquals(2, fixedInstance().without(IDENTIFIER1).getCommittedHydrants(IDENTIFIER2)); + Assert.assertEquals(0, fixedInstance().without(IDENTIFIER1).getCommittedPartialSegments(IDENTIFIER1)); + Assert.assertEquals(2, fixedInstance().without(IDENTIFIER1).getCommittedPartialSegments(IDENTIFIER2)); } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java index 538784a88539..d0d3d35b450b 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTest.java @@ -49,7 +49,7 @@ import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.realtime.sink.Committers; +import org.apache.druid.segment.realtime.Committers; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.DataSegment; @@ -311,7 +311,7 @@ public void run() } @Test - public void testMaxBytesInMemoryInMultipleSinksWithSkipBytesInMemoryOverheadCheckConfig() throws Exception + public void testMaxBytesInMemoryInMultipleAppendableSegmentsWithSkipBytesInMemoryOverheadCheckConfig() throws Exception { try ( final StreamAppenderatorTester tester = @@ -392,26 +392,26 @@ public void run() //expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 54 (dimsKeySize) = 182 + 1 byte when null handling is enabled int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; int currentInMemoryIndexSize = 182 + nullHandlingOverhead; - int sinkSizeOverhead = 1 * StreamAppenderator.ROUGH_OVERHEAD_PER_SINK; - // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet + int appendableSegmentSizeOverhead = 1 * StreamAppenderator.ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT; + // currPartialSegment in the AppendableSegment still has > 0 bytesInMemory since we do not persist yet Assert.assertEquals( currentInMemoryIndexSize, ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( - currentInMemoryIndexSize + sinkSizeOverhead, + currentInMemoryIndexSize + appendableSegmentSizeOverhead, ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); - // We do multiple more adds to the same sink to cause persist. + // We do multiple more adds to the same AppendableSegment to cause persist. for (int i = 0; i < 53; i++) { appenderator.add(IDENTIFIERS.get(0), ir("2000", "bar_" + i, 1), committerSupplier); } - sinkSizeOverhead = 1 * StreamAppenderator.ROUGH_OVERHEAD_PER_SINK; - // currHydrant size is 0 since we just persist all indexes to disk. + appendableSegmentSizeOverhead = 1 * StreamAppenderator.ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT; + // currPartialSegment size is 0 since we just persist all indexes to disk. currentInMemoryIndexSize = 0; // We are now over maxSizeInBytes after the add. Hence, we do a persist. - // currHydrant in the sink has 0 bytesInMemory since we just did a persist + // currPartialSegment in the AppendableSegment has 0 bytesInMemory since we just did a persist Assert.assertEquals( currentInMemoryIndexSize, ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) @@ -422,31 +422,31 @@ public void run() StreamAppenderator.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + StreamAppenderator.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER; Assert.assertEquals( - currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, + currentInMemoryIndexSize + appendableSegmentSizeOverhead + mappedIndexSize, ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); // Add a single row after persisted appenderator.add(IDENTIFIERS.get(0), ir("2000", "bob", 1), committerSupplier); - // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet + // currPartialSegment in the AppendableSegment still has > 0 bytesInMemory since we do not persist yet currentInMemoryIndexSize = 182 + nullHandlingOverhead; Assert.assertEquals( currentInMemoryIndexSize, ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) ); Assert.assertEquals( - currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, + currentInMemoryIndexSize + appendableSegmentSizeOverhead + mappedIndexSize, ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); - // We do multiple more adds to the same sink to cause persist. + // We do multiple more adds to the same AppendableSegment to cause persist. for (int i = 0; i < 31; i++) { appenderator.add(IDENTIFIERS.get(0), ir("2000", "bar_" + i, 1), committerSupplier); } - // currHydrant size is 0 since we just persist all indexes to disk. + // currPartialSegment size is 0 since we just persist all indexes to disk. currentInMemoryIndexSize = 0; // We are now over maxSizeInBytes after the add. Hence, we do a persist. - // currHydrant in the sink has 0 bytesInMemory since we just did a persist + // currPartialSegment in the AppendableSegment has 0 bytesInMemory since we just did a persist Assert.assertEquals( currentInMemoryIndexSize, ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) @@ -458,7 +458,7 @@ public void run() StreamAppenderator.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + StreamAppenderator.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER); Assert.assertEquals( - currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, + currentInMemoryIndexSize + appendableSegmentSizeOverhead + mappedIndexSize, ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); appenderator.close(); @@ -588,9 +588,9 @@ public void run() // Still under maxSizeInBytes after the add. Hence, we do not persist yet int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; int currentInMemoryIndexSize = 182 + nullHandlingOverhead; - int sinkSizeOverhead = 1 * StreamAppenderator.ROUGH_OVERHEAD_PER_SINK; + int appendableSegmentSizeOverhead = 1 * StreamAppenderator.ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT; Assert.assertEquals( - currentInMemoryIndexSize + sinkSizeOverhead, + currentInMemoryIndexSize + appendableSegmentSizeOverhead, ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); @@ -603,7 +603,7 @@ public void run() } @Test - public void testMaxBytesInMemoryInMultipleSinks() throws Exception + public void testMaxBytesInMemoryInMultipleAppendableSegments() throws Exception { try ( final StreamAppenderatorTester tester = @@ -640,8 +640,8 @@ public void run() //expectedSizeInBytes = 44(map overhead) + 28 (TimeAndDims overhead) + 56 (aggregator metrics) + 54 (dimsKeySize) = 182 + 1 byte when null handling is enabled int nullHandlingOverhead = NullHandling.sqlCompatible() ? 1 : 0; int currentInMemoryIndexSize = 182 + nullHandlingOverhead; - int sinkSizeOverhead = 2 * StreamAppenderator.ROUGH_OVERHEAD_PER_SINK; - // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet + int appendableSegmentSizeOverhead = 2 * StreamAppenderator.ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT; + // currPartialSegment in the AppendableSegment still has > 0 bytesInMemory since we do not persist yet Assert.assertEquals( currentInMemoryIndexSize, ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) @@ -651,20 +651,20 @@ public void run() ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); Assert.assertEquals( - (2 * currentInMemoryIndexSize) + sinkSizeOverhead, + (2 * currentInMemoryIndexSize) + appendableSegmentSizeOverhead, ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); - // We do multiple more adds to the same sink to cause persist. + // We do multiple more adds to the same AppendableSegment to cause persist. for (int i = 0; i < 49; i++) { appenderator.add(IDENTIFIERS.get(0), ir("2000", "bar_" + i, 1), committerSupplier); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar_" + i, 1), committerSupplier); } - sinkSizeOverhead = 2 * StreamAppenderator.ROUGH_OVERHEAD_PER_SINK; - // currHydrant size is 0 since we just persist all indexes to disk. + appendableSegmentSizeOverhead = 2 * StreamAppenderator.ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT; + // currPartialSegment size is 0 since we just persist all indexes to disk. currentInMemoryIndexSize = 0; // We are now over maxSizeInBytes after the add. Hence, we do a persist. - // currHydrant in the sink has 0 bytesInMemory since we just did a persist + // currPartialSegment in the AppendableSegment has 0 bytesInMemory since we just did a persist Assert.assertEquals( currentInMemoryIndexSize, ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) @@ -679,13 +679,13 @@ public void run() StreamAppenderator.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + StreamAppenderator.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER); Assert.assertEquals( - currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, + currentInMemoryIndexSize + appendableSegmentSizeOverhead + mappedIndexSize, ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); - // Add a single row after persisted to sink 0 + // Add a single row after persisted to AppendableSegment 0 appenderator.add(IDENTIFIERS.get(0), ir("2000", "bob", 1), committerSupplier); - // currHydrant in the sink still has > 0 bytesInMemory since we do not persist yet + // currPartialSegment in the AppendableSegment still has > 0 bytesInMemory since we do not persist yet currentInMemoryIndexSize = 182 + nullHandlingOverhead; Assert.assertEquals( currentInMemoryIndexSize, @@ -696,10 +696,10 @@ public void run() ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); Assert.assertEquals( - currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, + currentInMemoryIndexSize + appendableSegmentSizeOverhead + mappedIndexSize, ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); - // Now add a single row to sink 1 + // Now add a single row to AppendableSegment 1 appenderator.add(IDENTIFIERS.get(1), ir("2000", "bob", 1), committerSupplier); Assert.assertEquals( currentInMemoryIndexSize, @@ -710,19 +710,19 @@ public void run() ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(1)) ); Assert.assertEquals( - (2 * currentInMemoryIndexSize) + sinkSizeOverhead + mappedIndexSize, + (2 * currentInMemoryIndexSize) + appendableSegmentSizeOverhead + mappedIndexSize, ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); - // We do multiple more adds to the both sink to cause persist. + // We do multiple more adds to the both AppendableSegment to cause persist. for (int i = 0; i < 34; i++) { appenderator.add(IDENTIFIERS.get(0), ir("2000", "bar_" + i, 1), committerSupplier); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar_" + i, 1), committerSupplier); } - // currHydrant size is 0 since we just persist all indexes to disk. + // currPartialSegment size is 0 since we just persist all indexes to disk. currentInMemoryIndexSize = 0; // We are now over maxSizeInBytes after the add. Hence, we do a persist. - // currHydrant in the sink has 0 bytesInMemory since we just did a persist + // currPartialSegment in the AppendableSegment has 0 bytesInMemory since we just did a persist Assert.assertEquals( currentInMemoryIndexSize, ((StreamAppenderator) appenderator).getBytesInMemory(IDENTIFIERS.get(0)) @@ -738,7 +738,7 @@ public void run() StreamAppenderator.ROUGH_OVERHEAD_PER_DIMENSION_COLUMN_HOLDER + StreamAppenderator.ROUGH_OVERHEAD_PER_TIME_COLUMN_HOLDER)); Assert.assertEquals( - currentInMemoryIndexSize + sinkSizeOverhead + mappedIndexSize, + currentInMemoryIndexSize + appendableSegmentSizeOverhead + mappedIndexSize, ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); appenderator.close(); @@ -789,9 +789,9 @@ public void run() ); Assert.assertEquals(1, ((StreamAppenderator) appenderator).getRowsInMemory()); appenderator.add(IDENTIFIERS.get(1), ir("2000", "bar", 1), committerSupplier); - int sinkSizeOverhead = 2 * StreamAppenderator.ROUGH_OVERHEAD_PER_SINK; + int appendableSegmentSizeOverhead = 2 * StreamAppenderator.ROUGH_OVERHEAD_PER_APPENDABLE_SEGMENT; Assert.assertEquals( - (364 + 2 * nullHandlingOverhead) + sinkSizeOverhead, + (364 + 2 * nullHandlingOverhead) + appendableSegmentSizeOverhead, ((StreamAppenderator) appenderator).getBytesCurrentlyInMemory() ); Assert.assertEquals(2, ((StreamAppenderator) appenderator).getRowsInMemory()); @@ -1454,7 +1454,7 @@ public void testSchemaAnnouncement() throws Exception final ConcurrentMap commitMetadata = new ConcurrentHashMap<>(); final Supplier committerSupplier = committerSupplierFromConcurrentMap(commitMetadata); - StreamAppenderator.SinkSchemaAnnouncer sinkSchemaAnnouncer = appenderator.getSinkSchemaAnnouncer(); + StreamAppenderator.AppendableSegmentSchemaAnnouncer appendableSegmentSchemaAnnouncer = appenderator.getAppendableSegmentSchemaAnnouncer(); // startJob Assert.assertEquals(null, appenderator.startJob()); @@ -1467,7 +1467,7 @@ public void testSchemaAnnouncement() throws Exception appenderator.add(IDENTIFIERS.get(0), ir("2000", "foo", 1), committerSupplier); // trigger schema computation - sinkSchemaAnnouncer.computeAndAnnounce(); + appendableSegmentSchemaAnnouncer.computeAndAnnounce(); // verify schema List> announcedAbsoluteSchema = dataSegmentAnnouncer.getAnnouncedAbsoluteSchema(); @@ -1497,7 +1497,7 @@ public void testSchemaAnnouncement() throws Exception Assert.assertEquals(1, segmentSchemas.size()); Assert.assertEquals(IDENTIFIERS.get(0).asSegmentId().toString(), deltaSchemaId1Row1.getSegmentId()); Assert.assertEquals(1, deltaSchemaId1Row1.getNumRows().intValue()); - // absolute schema is sent for a new sink + // absolute schema is sent for a new AppendableSegment Assert.assertFalse(deltaSchemaId1Row1.isDelta()); Assert.assertEquals(Collections.emptyList(), deltaSchemaId1Row1.getUpdatedColumns()); Assert.assertEquals(Lists.newArrayList("__time", "dim", "count", "met"), deltaSchemaId1Row1.getNewColumns()); @@ -1512,7 +1512,7 @@ public void testSchemaAnnouncement() throws Exception appenderator.add(IDENTIFIERS.get(0), ir("2000", "bar", 2), committerSupplier); // trigger schema computation - sinkSchemaAnnouncer.computeAndAnnounce(); + appendableSegmentSchemaAnnouncer.computeAndAnnounce(); // verify schema announcedAbsoluteSchema = dataSegmentAnnouncer.getAnnouncedAbsoluteSchema(); @@ -1553,7 +1553,7 @@ public void testSchemaAnnouncement() throws Exception commitMetadata.put("x", "3"); appenderator.add(IDENTIFIERS.get(1), ir("2000", "qux", 4), committerSupplier); - sinkSchemaAnnouncer.computeAndAnnounce(); + appendableSegmentSchemaAnnouncer.computeAndAnnounce(); // verify schema announcedAbsoluteSchema = dataSegmentAnnouncer.getAnnouncedAbsoluteSchema(); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseIdleTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseIdleTest.java deleted file mode 100644 index 419b29ace7e2..000000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseIdleTest.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.segment.realtime.firehose; - -import com.google.common.collect.ImmutableList; -import org.apache.commons.io.IOUtils; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.server.metrics.EventReceiverFirehoseRegister; -import org.apache.druid.server.security.AllowAllAuthenticator; -import org.apache.druid.server.security.AuthConfig; -import org.apache.druid.server.security.AuthTestUtils; -import org.easymock.EasyMock; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import javax.servlet.http.HttpServletRequest; -import java.util.Locale; - -public class EventReceiverFirehoseIdleTest -{ - private static final int CAPACITY = 300; - private static final long MAX_IDLE_TIME = 5_000L; - private static final String SERVICE_NAME = "test_firehose"; - - private final String inputRow = "[{\n" - + " \"timestamp\":123,\n" - + " \"d1\":\"v1\"\n" - + "}]"; - - private EventReceiverFirehoseFactory eventReceiverFirehoseFactory; - private EventReceiverFirehoseFactory.EventReceiverFirehose firehose; - private EventReceiverFirehoseRegister register = new EventReceiverFirehoseRegister(); - private HttpServletRequest req; - - @Before - public void setUp() - { - req = EasyMock.createMock(HttpServletRequest.class); - eventReceiverFirehoseFactory = new EventReceiverFirehoseFactory( - SERVICE_NAME, - CAPACITY, - MAX_IDLE_TIME, - null, - new DefaultObjectMapper(), - new DefaultObjectMapper(), - register, - AuthTestUtils.TEST_AUTHORIZER_MAPPER - ); - firehose = (EventReceiverFirehoseFactory.EventReceiverFirehose) eventReceiverFirehoseFactory.connect( - new MapInputRowParser( - new JSONParseSpec( - new TimestampSpec( - "timestamp", - "auto", - null - ), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("d1"))), - null, - null, - null - ) - ), - null - ); - } - - @Test(timeout = 40_000L) - public void testIdle() throws Exception - { - awaitFirehoseClosed(); - awaitDelayedExecutorThreadTerminated(); - } - - private void awaitFirehoseClosed() throws InterruptedException - { - while (!firehose.isClosed()) { - Thread.sleep(50); - } - } - - private void awaitDelayedExecutorThreadTerminated() throws InterruptedException - { - firehose.getDelayedCloseExecutor().join(); - } - - @Test(timeout = 40_000L) - public void testNotIdle() throws Exception - { - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) - .andReturn(null) - .anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)) - .andReturn(null) - .anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) - .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) - .anyTimes(); - EasyMock.expect(req.getHeader("X-Firehose-Producer-Id")).andReturn(null).anyTimes(); - EasyMock.expect(req.getContentType()).andReturn("application/json").anyTimes(); - req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); - EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(req); - - final int checks = 5; - for (int i = 0; i < checks; i++) { - Assert.assertFalse(firehose.isClosed()); - System.out.printf(Locale.ENGLISH, "Check %d/%d passed\n", i + 1, checks); - firehose.addAll(IOUtils.toInputStream(inputRow), req); - Thread.sleep(3_000L); - } - - awaitFirehoseClosed(); - awaitDelayedExecutorThreadTerminated(); - } -} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java deleted file mode 100644 index 38b16c79cab4..000000000000 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java +++ /dev/null @@ -1,442 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.segment.realtime.firehose; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; -import org.apache.commons.io.IOUtils; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.server.metrics.EventReceiverFirehoseMetric; -import org.apache.druid.server.metrics.EventReceiverFirehoseRegister; -import org.apache.druid.server.security.AllowAllAuthenticator; -import org.apache.druid.server.security.AuthConfig; -import org.apache.druid.server.security.AuthTestUtils; -import org.easymock.EasyMock; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.core.Response; -import java.io.IOException; -import java.io.InputStream; -import java.nio.charset.StandardCharsets; -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - -public class EventReceiverFirehoseTest -{ - private static final int CAPACITY = 300; - private static final int NUM_EVENTS = 100; - private static final long MAX_IDLE_TIME_MILLIS = TimeUnit.SECONDS.toMillis(20); - private static final String SERVICE_NAME = "test_firehose"; - - private final String inputRow = "[{\n" - + " \"timestamp\":123,\n" - + " \"d1\":\"v1\"\n" - + "}]"; - - private EventReceiverFirehoseFactory eventReceiverFirehoseFactory; - private EventReceiverFirehoseFactory.EventReceiverFirehose firehose; - private EventReceiverFirehoseRegister register = new EventReceiverFirehoseRegister(); - private HttpServletRequest req; - - @Before - public void setUp() - { - req = EasyMock.createMock(HttpServletRequest.class); - eventReceiverFirehoseFactory = new EventReceiverFirehoseFactory( - SERVICE_NAME, - CAPACITY, - MAX_IDLE_TIME_MILLIS, - null, - new DefaultObjectMapper(), - new DefaultObjectMapper(), - register, - AuthTestUtils.TEST_AUTHORIZER_MAPPER - ); - firehose = (EventReceiverFirehoseFactory.EventReceiverFirehose) eventReceiverFirehoseFactory.connect( - new MapInputRowParser( - new JSONParseSpec( - new TimestampSpec( - "timestamp", - "auto", - null - ), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("d1"))), - null, - null, - null - ) - ), - null - ); - } - - @Test(timeout = 60_000L) - public void testSingleThread() throws IOException, InterruptedException - { - for (int i = 0; i < NUM_EVENTS; ++i) { - setUpRequestExpectations(null, null); - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - firehose.addAll(inputStream, req); - Assert.assertEquals(i + 1, firehose.getCurrentBufferSize()); - inputStream.close(); - } - - EasyMock.verify(req); - - final Iterable> metrics = register.getMetrics(); - Assert.assertEquals(1, Iterables.size(metrics)); - - final Map.Entry entry = Iterables.getLast(metrics); - Assert.assertEquals(SERVICE_NAME, entry.getKey()); - Assert.assertEquals(CAPACITY, entry.getValue().getCapacity()); - Assert.assertEquals(CAPACITY, firehose.getCapacity()); - Assert.assertEquals(NUM_EVENTS, entry.getValue().getCurrentBufferSize()); - Assert.assertEquals(NUM_EVENTS, firehose.getCurrentBufferSize()); - - for (int i = NUM_EVENTS - 1; i >= 0; --i) { - Assert.assertTrue(firehose.hasMore()); - Assert.assertNotNull(firehose.nextRow()); - Assert.assertEquals(i, firehose.getCurrentBufferSize()); - } - - Assert.assertEquals(CAPACITY, entry.getValue().getCapacity()); - Assert.assertEquals(CAPACITY, firehose.getCapacity()); - Assert.assertEquals(0, entry.getValue().getCurrentBufferSize()); - Assert.assertEquals(0, firehose.getCurrentBufferSize()); - - firehose.close(); - Assert.assertFalse(firehose.hasMore()); - Assert.assertEquals(0, Iterables.size(register.getMetrics())); - - awaitDelayedExecutorThreadTerminated(); - } - - @Test(timeout = 60_000L) - public void testMultipleThreads() throws InterruptedException, IOException, TimeoutException, ExecutionException - { - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) - .andReturn(null) - .anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) - .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) - .anyTimes(); - req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); - EasyMock.expectLastCall().anyTimes(); - - EasyMock.expect(req.getContentType()).andReturn("application/json").times(2 * NUM_EVENTS); - EasyMock.expect(req.getHeader("X-Firehose-Producer-Id")).andReturn(null).times(2 * NUM_EVENTS); - EasyMock.replay(req); - - final ExecutorService executorService = Execs.singleThreaded("single_thread"); - final Future future = executorService.submit( - new Callable() - { - @Override - public Boolean call() throws Exception - { - for (int i = 0; i < NUM_EVENTS; ++i) { - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - firehose.addAll(inputStream, req); - inputStream.close(); - } - return true; - } - } - ); - - for (int i = 0; i < NUM_EVENTS; ++i) { - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - firehose.addAll(inputStream, req); - inputStream.close(); - } - - future.get(10, TimeUnit.SECONDS); - - EasyMock.verify(req); - - final Iterable> metrics = register.getMetrics(); - Assert.assertEquals(1, Iterables.size(metrics)); - - final Map.Entry entry = Iterables.getLast(metrics); - - Assert.assertEquals(SERVICE_NAME, entry.getKey()); - Assert.assertEquals(CAPACITY, entry.getValue().getCapacity()); - Assert.assertEquals(CAPACITY, firehose.getCapacity()); - Assert.assertEquals(2 * NUM_EVENTS, entry.getValue().getCurrentBufferSize()); - Assert.assertEquals(2 * NUM_EVENTS, firehose.getCurrentBufferSize()); - - for (int i = 2 * NUM_EVENTS - 1; i >= 0; --i) { - Assert.assertTrue(firehose.hasMore()); - Assert.assertNotNull(firehose.nextRow()); - Assert.assertEquals(i, firehose.getCurrentBufferSize()); - } - - Assert.assertEquals(CAPACITY, entry.getValue().getCapacity()); - Assert.assertEquals(CAPACITY, firehose.getCapacity()); - Assert.assertEquals(0, entry.getValue().getCurrentBufferSize()); - Assert.assertEquals(0, firehose.getCurrentBufferSize()); - - firehose.close(); - Assert.assertFalse(firehose.hasMore()); - Assert.assertEquals(0, Iterables.size(register.getMetrics())); - - awaitDelayedExecutorThreadTerminated(); - - executorService.shutdownNow(); - } - - @Test(expected = ISE.class) - public void testDuplicateRegistering() - { - EventReceiverFirehoseFactory eventReceiverFirehoseFactory2 = new EventReceiverFirehoseFactory( - SERVICE_NAME, - CAPACITY, - MAX_IDLE_TIME_MILLIS, - null, - new DefaultObjectMapper(), - new DefaultObjectMapper(), - register, - AuthTestUtils.TEST_AUTHORIZER_MAPPER - ); - EventReceiverFirehoseFactory.EventReceiverFirehose firehose2 = - (EventReceiverFirehoseFactory.EventReceiverFirehose) eventReceiverFirehoseFactory2 - .connect( - new MapInputRowParser( - new JSONParseSpec( - new TimestampSpec( - "timestamp", - "auto", - null - ), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("d1"))), - null, - null, - null - ) - ), - null - ); - } - - @Test(timeout = 60_000L) - public void testShutdownWithPrevTime() throws Exception - { - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) - .andReturn(null) - .anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) - .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) - .anyTimes(); - req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); - EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(req); - - firehose.shutdown(DateTimes.nowUtc().minusMinutes(2).toString(), req); - awaitFirehoseClosed(); - awaitDelayedExecutorThreadTerminated(); - } - - private void awaitFirehoseClosed() throws InterruptedException - { - while (!firehose.isClosed()) { - Thread.sleep(50); - } - } - - private void awaitDelayedExecutorThreadTerminated() throws InterruptedException - { - firehose.getDelayedCloseExecutor().join(); - } - - @Test(timeout = 60_000L) - public void testShutdown() throws Exception - { - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) - .andReturn(null) - .anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) - .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) - .anyTimes(); - req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); - EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(req); - - firehose.shutdown(DateTimes.nowUtc().plusMillis(100).toString(), req); - awaitFirehoseClosed(); - awaitDelayedExecutorThreadTerminated(); - } - - @Test - public void testProducerSequence() throws IOException - { - for (int i = 0; i < NUM_EVENTS; ++i) { - setUpRequestExpectations("producer", String.valueOf(i)); - - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - firehose.addAll(inputStream, req); - Assert.assertEquals(i + 1, firehose.getCurrentBufferSize()); - inputStream.close(); - } - - EasyMock.verify(req); - - final Iterable> metrics = register.getMetrics(); - Assert.assertEquals(1, Iterables.size(metrics)); - - final Map.Entry entry = Iterables.getLast(metrics); - Assert.assertEquals(SERVICE_NAME, entry.getKey()); - Assert.assertEquals(CAPACITY, entry.getValue().getCapacity()); - Assert.assertEquals(CAPACITY, firehose.getCapacity()); - Assert.assertEquals(NUM_EVENTS, entry.getValue().getCurrentBufferSize()); - Assert.assertEquals(NUM_EVENTS, firehose.getCurrentBufferSize()); - - for (int i = NUM_EVENTS - 1; i >= 0; --i) { - Assert.assertTrue(firehose.hasMore()); - Assert.assertNotNull(firehose.nextRow()); - Assert.assertEquals(i, firehose.getCurrentBufferSize()); - } - - Assert.assertEquals(CAPACITY, entry.getValue().getCapacity()); - Assert.assertEquals(CAPACITY, firehose.getCapacity()); - Assert.assertEquals(0, entry.getValue().getCurrentBufferSize()); - Assert.assertEquals(0, firehose.getCurrentBufferSize()); - - firehose.close(); - Assert.assertFalse(firehose.hasMore()); - Assert.assertEquals(0, Iterables.size(register.getMetrics())); - } - - @Test - public void testLowProducerSequence() throws IOException - { - for (int i = 0; i < NUM_EVENTS; ++i) { - setUpRequestExpectations("producer", "1"); - - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - final Response response = firehose.addAll(inputStream, req); - Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertEquals(1, firehose.getCurrentBufferSize()); - inputStream.close(); - } - - EasyMock.verify(req); - - firehose.close(); - } - - @Test - public void testMissingProducerSequence() throws IOException - { - setUpRequestExpectations("producer", null); - - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - final Response response = firehose.addAll(inputStream, req); - - Assert.assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), response.getStatus()); - - inputStream.close(); - - EasyMock.verify(req); - - firehose.close(); - } - - @Test - public void testTooManyProducerIds() throws IOException - { - for (int i = 0; i < EventReceiverFirehoseFactory.MAX_FIREHOSE_PRODUCERS - 1; i++) { - setUpRequestExpectations("producer-" + i, "0"); - - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - final Response response = firehose.addAll(inputStream, req); - Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - inputStream.close(); - Assert.assertTrue(firehose.hasMore()); - Assert.assertNotNull(firehose.nextRow()); - } - - setUpRequestExpectations("toomany", "0"); - - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - final Response response = firehose.addAll(inputStream, req); - Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), response.getStatus()); - inputStream.close(); - - EasyMock.verify(req); - - firehose.close(); - } - - @Test - public void testNaNProducerSequence() throws IOException - { - setUpRequestExpectations("producer", "foo"); - - final InputStream inputStream = IOUtils.toInputStream(inputRow, StandardCharsets.UTF_8); - final Response response = firehose.addAll(inputStream, req); - - Assert.assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), response.getStatus()); - - inputStream.close(); - - EasyMock.verify(req); - - firehose.close(); - } - - private void setUpRequestExpectations(String producerId, String producerSequenceValue) - { - EasyMock.reset(req); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) - .andReturn(null) - .anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) - .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) - .anyTimes(); - req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); - EasyMock.expectLastCall().anyTimes(); - - EasyMock.expect(req.getContentType()).andReturn("application/json"); - EasyMock.expect(req.getHeader("X-Firehose-Producer-Id")).andReturn(producerId); - - if (producerId != null) { - EasyMock.expect(req.getHeader("X-Firehose-Producer-Seq")).andReturn(producerSequenceValue); - } - - EasyMock.replay(req); - } -} diff --git a/services/src/main/java/org/apache/druid/cli/CliIndexer.java b/services/src/main/java/org/apache/druid/cli/CliIndexer.java index c6b817fa4a9f..50e0e299748a 100644 --- a/services/src/main/java/org/apache/druid/cli/CliIndexer.java +++ b/services/src/main/java/org/apache/druid/cli/CliIndexer.java @@ -36,7 +36,6 @@ import org.apache.druid.discovery.WorkerNodeService; import org.apache.druid.guice.DruidProcessingModule; import org.apache.druid.guice.IndexerServiceModule; -import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.IndexingServiceModuleHelper; import org.apache.druid.guice.IndexingServiceTaskLogsModule; @@ -236,7 +235,6 @@ public DataNodeService getDataNodeService(DruidServerConfig serverConfig) } }, new ShuffleModule(), - new IndexingServiceFirehoseModule(), new IndexingServiceInputSourceModule(), new IndexingServiceTaskLogsModule(), new IndexingServiceTuningConfigModule(), diff --git a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java index 37b6501bf87f..2e542bc69745 100644 --- a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java @@ -36,7 +36,6 @@ import org.apache.druid.curator.ZkEnablementConfig; import org.apache.druid.discovery.NodeRole; import org.apache.druid.discovery.WorkerNodeService; -import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.IndexingServiceModuleHelper; import org.apache.druid.guice.IndexingServiceTaskLogsModule; @@ -73,10 +72,10 @@ import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.http.SelfDiscoveryResource; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; @@ -242,7 +241,6 @@ public WorkerNodeService getWorkerNodeService(WorkerConfig workerConfig) } }, new ShuffleModule(), - new IndexingServiceFirehoseModule(), new IndexingServiceInputSourceModule(), new IndexingServiceTaskLogsModule(), new IndexingServiceTuningConfigModule(), diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 24e98427ce95..9ba14dec9322 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -39,7 +39,6 @@ import com.google.inject.util.Providers; import org.apache.druid.client.indexing.IndexingService; import org.apache.druid.discovery.NodeRole; -import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.IndexingServiceModuleHelper; import org.apache.druid.guice.IndexingServiceTaskLogsModule; @@ -108,10 +107,10 @@ import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.http.RedirectFilter; import org.apache.druid.server.http.RedirectInfo; @@ -428,7 +427,6 @@ private void configureOverlordHelpers(Binder binder) .to(TaskLogAutoCleaner.class); } }, - new IndexingServiceFirehoseModule(), new IndexingServiceInputSourceModule(), new IndexingServiceTaskLogsModule(), new IndexingServiceTuningConfigModule(), diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index 1ca8ddf539fc..375940a49b08 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -48,7 +48,6 @@ import org.apache.druid.guice.Binders; import org.apache.druid.guice.CacheModule; import org.apache.druid.guice.DruidProcessingModule; -import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.IndexingServiceTaskLogsModule; import org.apache.druid.guice.IndexingServiceTuningConfigModule; @@ -117,11 +116,11 @@ import org.apache.druid.segment.loading.OmniDataSegmentMover; import org.apache.druid.segment.loading.StorageLocation; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.realtime.ChatHandlerProvider; +import org.apache.druid.segment.realtime.NoopChatHandlerProvider; +import org.apache.druid.segment.realtime.ServiceAnnouncingChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.PeonAppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.ServiceAnnouncingChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.ResponseContextConfig; import org.apache.druid.server.SegmentManager; @@ -343,7 +342,6 @@ public LookupLoadingSpec getLookupsToLoad(final Task task) } }, new QueryablePeonModule(), - new IndexingServiceFirehoseModule(), new IndexingServiceInputSourceModule(), new IndexingServiceTuningConfigModule(), new InputSourceModule(), diff --git a/services/src/main/java/org/apache/druid/cli/validate/DruidJsonValidator.java b/services/src/main/java/org/apache/druid/cli/validate/DruidJsonValidator.java index 0518c7259e63..50684cee0e87 100644 --- a/services/src/main/java/org/apache/druid/cli/validate/DruidJsonValidator.java +++ b/services/src/main/java/org/apache/druid/cli/validate/DruidJsonValidator.java @@ -40,8 +40,6 @@ import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.guice.DruidProcessingModule; import org.apache.druid.guice.ExtensionsLoader; -import org.apache.druid.guice.FirehoseModule; -import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.LocalDataStorageDruidModule; import org.apache.druid.guice.QueryRunnerFactoryModule; @@ -130,9 +128,7 @@ public void run() Iterables.concat( extnLoader.getModules(), Arrays.asList( - new FirehoseModule(), new IndexingHadoopModule(), - new IndexingServiceFirehoseModule(), new IndexingServiceInputSourceModule(), new LocalDataStorageDruidModule() ) diff --git a/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java b/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java index b843465147eb..b617b7e6b877 100644 --- a/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java @@ -142,7 +142,6 @@ public void testTaskValidator() throws Exception jsonMapper ), new IndexTask.IndexIOConfig( - null, new LocalInputSource(new File("lol"), "rofl"), new JsonInputFormat(null, null, null, null, null), false, diff --git a/sql/src/test/resources/drill/window/queries/multiple_partitions/q26.sql b/sql/src/test/resources/drill/window/queries/multiple_partitions/q26.sql index 03bcec6e410f..c53dc6d27006 100644 --- a/sql/src/test/resources/drill/window/queries/multiple_partitions/q26.sql +++ b/sql/src/test/resources/drill/window/queries/multiple_partitions/q26.sql @@ -1,4 +1,4 @@ --- Kitchen sink +-- Kitchen appendableSegment -- Use all supported functions select rank() over W, diff --git a/sql/src/test/resources/drill/window/queries/multiple_partitions/q27.sql b/sql/src/test/resources/drill/window/queries/multiple_partitions/q27.sql index 6bb885d527c4..9fa72d64696c 100644 --- a/sql/src/test/resources/drill/window/queries/multiple_partitions/q27.sql +++ b/sql/src/test/resources/drill/window/queries/multiple_partitions/q27.sql @@ -1,4 +1,4 @@ --- Kitchen sink +-- Kitchen appendableSegment -- Use all supported functions select rank() over W,