mirror of https://github.com/apache/druid.git
Remove index_realtime and index_realtime_appenderator tasks (#16602)
index_realtime tasks were removed from the documentation in #13107. Even at that time, they weren't really documented per se— just mentioned. They existed solely to support Tranquility, which is an obsolete ingestion method that predates migration of Druid to ASF and is no longer being maintained. Tranquility docs were also de-linked from the sidebars and the other doc pages in #11134. Only a stub remains, so people with links to the page can see that it's no longer recommended. index_realtime_appenderator tasks existed in the code base, but were never documented, nor as far as I am aware were they used for any purpose. This patch removes both task types completely, as well as removes all supporting code that was otherwise unused. It also updates the stub doc for Tranquility to be firmer that it is not compatible. (Previously, the stub doc said it wasn't recommended, and pointed out that it is built against an ancient 0.9.2 version of Druid.) ITUnionQueryTest has been migrated to the new integration tests framework and updated to use Kafka ingestion. Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
This commit is contained in:
parent
2131917f16
commit
37a50e6803
|
@ -50,7 +50,7 @@ jobs:
|
|||
matrix:
|
||||
#jdk: [8, 11, 17]
|
||||
jdk: [8]
|
||||
it: [HighAvailability, MultiStageQuery, Catalog, BatchIndex, MultiStageQueryWithMM, InputSource, InputFormat, Security]
|
||||
it: [HighAvailability, MultiStageQuery, Catalog, BatchIndex, MultiStageQueryWithMM, InputSource, InputFormat, Security, Query]
|
||||
#indexer: [indexer, middleManager]
|
||||
indexer: [middleManager]
|
||||
uses: ./.github/workflows/reusable-revised-its.yml
|
||||
|
|
|
@ -16,10 +16,9 @@
|
|||
# limitations under the License.
|
||||
|
||||
import os
|
||||
import requests
|
||||
import subprocess
|
||||
import sys
|
||||
import requests
|
||||
|
||||
|
||||
if len(sys.argv) != 5:
|
||||
sys.stderr.write('usage: program <github-username> <previous-release-commit> <new-release-commit> <milestone-number-to-tag>\n')
|
||||
|
|
|
@ -22,15 +22,9 @@ title: "Tranquility"
|
|||
~ under the License.
|
||||
-->
|
||||
|
||||
[Tranquility](https://github.com/druid-io/tranquility/) is a separately distributed package for pushing
|
||||
streams to Druid in real-time.
|
||||
|
||||
Tranquility has not been built against a version of Druid later than Druid 0.9.2
|
||||
release. It may still work with the latest Druid servers, but not all features and functionality will be available
|
||||
due to limitations of older Druid APIs on the Tranquility side.
|
||||
[Tranquility](https://github.com/druid-io/tranquility/) was a separately distributed package for pushing
|
||||
streams to Druid in real-time. It is not compatible with recent versions of Druid.
|
||||
|
||||
For new projects that require streaming ingestion, we recommend using Druid's native support for
|
||||
[Apache Kafka](../ingestion/kafka-ingestion.md) or
|
||||
[Amazon Kinesis](../ingestion/kinesis-ingestion.md).
|
||||
|
||||
For more details, check out the [Tranquility GitHub page](https://github.com/druid-io/tranquility/).
|
||||
|
|
|
@ -64,7 +64,6 @@ import org.junit.Before;
|
|||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
|
@ -145,7 +144,7 @@ public class MaterializedViewSupervisorTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testCheckSegments() throws IOException
|
||||
public void testCheckSegments()
|
||||
{
|
||||
List<DataSegment> baseSegments = createBaseSegments();
|
||||
Set<DataSegment> derivativeSegments = Sets.newHashSet(createDerivativeSegments());
|
||||
|
@ -171,7 +170,7 @@ public class MaterializedViewSupervisorTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testSubmitTasksDoesNotFailIfTaskAlreadyExists() throws IOException
|
||||
public void testSubmitTasksDoesNotFailIfTaskAlreadyExists()
|
||||
{
|
||||
Set<DataSegment> baseSegments = Sets.newHashSet(createBaseSegments());
|
||||
Set<DataSegment> derivativeSegments = Sets.newHashSet(createDerivativeSegments());
|
||||
|
@ -193,7 +192,7 @@ public class MaterializedViewSupervisorTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testSubmitTasksFailsIfTaskCannotBeAdded() throws IOException
|
||||
public void testSubmitTasksFailsIfTaskCannotBeAdded()
|
||||
{
|
||||
Set<DataSegment> baseSegments = Sets.newHashSet(createBaseSegments());
|
||||
Set<DataSegment> derivativeSegments = Sets.newHashSet(createDerivativeSegments());
|
||||
|
@ -219,7 +218,7 @@ public class MaterializedViewSupervisorTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testCheckSegmentsAndSubmitTasks() throws IOException
|
||||
public void testCheckSegmentsAndSubmitTasks()
|
||||
{
|
||||
Set<DataSegment> baseSegments = Collections.singleton(createBaseSegments().get(0));
|
||||
indexerMetadataStorageCoordinator.commitSegments(baseSegments, null);
|
||||
|
|
|
@ -178,13 +178,8 @@ public class DatasourceOptimizerTest extends CuratorTestBase
|
|||
Lists.newArrayList("dim1", "dim2", "dim3", "dim4"),
|
||||
1024 * 1024
|
||||
);
|
||||
try {
|
||||
metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment), null);
|
||||
announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper);
|
||||
}
|
||||
catch (IOException e) {
|
||||
return false;
|
||||
}
|
||||
metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment), null);
|
||||
announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper);
|
||||
return true;
|
||||
}
|
||||
);
|
||||
|
@ -203,13 +198,8 @@ public class DatasourceOptimizerTest extends CuratorTestBase
|
|||
Lists.newArrayList("dim1", "dim2", "dim3"),
|
||||
1024
|
||||
);
|
||||
try {
|
||||
metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment), null);
|
||||
announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper);
|
||||
}
|
||||
catch (IOException e) {
|
||||
return false;
|
||||
}
|
||||
metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment), null);
|
||||
announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper);
|
||||
return true;
|
||||
}
|
||||
);
|
||||
|
|
|
@ -49,7 +49,6 @@ import org.junit.Assert;
|
|||
import org.junit.Test;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
|
|
|
@ -41,7 +41,6 @@ import javax.servlet.ServletRequest;
|
|||
import javax.servlet.ServletResponse;
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import javax.servlet.http.HttpServletResponse;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
|
||||
|
|
|
@ -367,7 +367,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
// Wait for task to exit
|
||||
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
|
||||
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3));
|
||||
Assert.assertTrue(task.getRunner().getFireDepartmentMetrics().isProcessingDone());
|
||||
Assert.assertTrue(task.getRunner().getSegmentGenerationMetrics().isProcessingDone());
|
||||
|
||||
// Check published metadata and segments in deep storage
|
||||
assertEqualsExceptVersion(
|
||||
|
@ -505,7 +505,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
// Wait for task to exit
|
||||
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
|
||||
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3));
|
||||
Assert.assertTrue(task.getRunner().getFireDepartmentMetrics().isProcessingDone());
|
||||
Assert.assertTrue(task.getRunner().getSegmentGenerationMetrics().isProcessingDone());
|
||||
|
||||
// Check published metadata and segments in deep storage
|
||||
assertEqualsExceptVersion(
|
||||
|
@ -554,7 +554,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
// Wait for task to exit
|
||||
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
|
||||
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3));
|
||||
Assert.assertTrue(task.getRunner().getFireDepartmentMetrics().isProcessingDone());
|
||||
Assert.assertTrue(task.getRunner().getSegmentGenerationMetrics().isProcessingDone());
|
||||
|
||||
// Check published metadata and segments in deep storage
|
||||
assertEqualsExceptVersion(
|
||||
|
|
|
@ -40,7 +40,7 @@ import org.apache.druid.indexer.TaskLocation;
|
|||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.TaskInfoProvider;
|
||||
import org.apache.druid.indexing.common.TestUtils;
|
||||
import org.apache.druid.indexing.common.task.RealtimeIndexTask;
|
||||
import org.apache.druid.indexing.common.task.NoopTask;
|
||||
import org.apache.druid.indexing.common.task.Task;
|
||||
import org.apache.druid.indexing.kafka.KafkaConsumerConfigs;
|
||||
import org.apache.druid.indexing.kafka.KafkaDataSourceMetadata;
|
||||
|
@ -89,9 +89,7 @@ import org.apache.druid.segment.TestHelper;
|
|||
import org.apache.druid.segment.incremental.ParseExceptionReport;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
import org.apache.druid.segment.realtime.FireDepartment;
|
||||
import org.apache.druid.server.metrics.NoopServiceEmitter;
|
||||
import org.apache.druid.server.security.Action;
|
||||
import org.apache.druid.server.security.Resource;
|
||||
|
@ -1194,15 +1192,13 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
addSomeEvents(1);
|
||||
|
||||
// non KafkaIndexTask (don't kill)
|
||||
Task id2 = new RealtimeIndexTask(
|
||||
Task id2 = new NoopTask(
|
||||
"id2",
|
||||
null,
|
||||
new FireDepartment(
|
||||
dataSchema,
|
||||
new RealtimeIOConfig(null, null),
|
||||
null
|
||||
),
|
||||
null
|
||||
dataSchema.getDataSource(),
|
||||
100,
|
||||
100,
|
||||
ImmutableMap.of()
|
||||
);
|
||||
|
||||
List<Task> existingTasks = ImmutableList.of(id2);
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.druid.indexer.TaskLocation;
|
|||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.TaskInfoProvider;
|
||||
import org.apache.druid.indexing.common.TestUtils;
|
||||
import org.apache.druid.indexing.common.task.RealtimeIndexTask;
|
||||
import org.apache.druid.indexing.common.task.NoopTask;
|
||||
import org.apache.druid.indexing.common.task.Task;
|
||||
import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata;
|
||||
import org.apache.druid.indexing.kinesis.KinesisIndexTask;
|
||||
|
@ -84,9 +84,7 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
|||
import org.apache.druid.segment.TestHelper;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
import org.apache.druid.segment.realtime.FireDepartment;
|
||||
import org.apache.druid.server.metrics.NoopServiceEmitter;
|
||||
import org.apache.druid.server.security.Action;
|
||||
import org.apache.druid.server.security.Resource;
|
||||
|
@ -940,15 +938,13 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
EasyMock.expectLastCall().anyTimes();
|
||||
|
||||
// non KinesisIndexTask (don't kill)
|
||||
Task id2 = new RealtimeIndexTask(
|
||||
Task id2 = new NoopTask(
|
||||
"id2",
|
||||
null,
|
||||
new FireDepartment(
|
||||
dataSchema,
|
||||
new RealtimeIOConfig(null, null),
|
||||
null
|
||||
),
|
||||
null
|
||||
dataSchema.getDataSource(),
|
||||
100,
|
||||
100,
|
||||
ImmutableMap.of()
|
||||
);
|
||||
|
||||
List<Task> existingTasks = ImmutableList.of(id2);
|
||||
|
|
|
@ -20,15 +20,15 @@
|
|||
package org.apache.druid.msq.counters;
|
||||
|
||||
import org.apache.druid.msq.indexing.processor.SegmentGeneratorFrameProcessor;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||
import org.apache.druid.segment.realtime.SegmentGenerationMetrics;
|
||||
|
||||
/**
|
||||
* Wrapper around {@link FireDepartmentMetrics} which updates the progress counters while updating its metrics. This
|
||||
* Wrapper around {@link SegmentGenerationMetrics} which updates the progress counters while updating its metrics. This
|
||||
* is necessary as the {@link org.apache.druid.segment.realtime.appenderator.BatchAppenderator} used by the
|
||||
* {@link SegmentGeneratorFrameProcessor} is not part of the MSQ extension, and hence,
|
||||
* cannot update the counters used in MSQ reports as it persists and pushes segments to deep storage.
|
||||
*/
|
||||
public class SegmentGeneratorMetricsWrapper extends FireDepartmentMetrics
|
||||
public class SegmentGeneratorMetricsWrapper extends SegmentGenerationMetrics
|
||||
{
|
||||
private final SegmentGenerationProgressCounter segmentGenerationProgressCounter;
|
||||
|
||||
|
|
|
@ -88,7 +88,6 @@ import org.joda.time.Interval;
|
|||
import org.mockito.Mockito;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
|
|
@ -19,14 +19,12 @@
|
|||
|
||||
package org.apache.druid.indexing.common;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor;
|
||||
import org.apache.druid.indexing.common.task.Task;
|
||||
import org.apache.druid.query.DruidMetrics;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||
import org.apache.druid.segment.realtime.FireDepartment;
|
||||
import org.apache.druid.segment.realtime.RealtimeMetricsMonitor;
|
||||
import org.apache.druid.segment.realtime.SegmentGenerationMetrics;
|
||||
|
||||
public class TaskRealtimeMetricsMonitorBuilder
|
||||
{
|
||||
|
@ -34,31 +32,21 @@ public class TaskRealtimeMetricsMonitorBuilder
|
|||
{
|
||||
}
|
||||
|
||||
public static RealtimeMetricsMonitor build(Task task, FireDepartment fireDepartment)
|
||||
{
|
||||
return new RealtimeMetricsMonitor(
|
||||
ImmutableList.of(fireDepartment),
|
||||
ImmutableMap.of(
|
||||
DruidMetrics.TASK_ID, new String[]{task.getId()},
|
||||
DruidMetrics.TASK_TYPE, new String[]{task.getType()}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
public static TaskRealtimeMetricsMonitor build(
|
||||
Task task,
|
||||
FireDepartment fireDepartment,
|
||||
SegmentGenerationMetrics metrics,
|
||||
RowIngestionMeters meters
|
||||
)
|
||||
{
|
||||
return new TaskRealtimeMetricsMonitor(
|
||||
fireDepartment,
|
||||
metrics,
|
||||
meters,
|
||||
ImmutableMap.of(
|
||||
DruidMetrics.DATASOURCE, new String[]{task.getDataSource()},
|
||||
DruidMetrics.TASK_ID, new String[]{task.getId()},
|
||||
DruidMetrics.TASK_TYPE, new String[]{task.getType()},
|
||||
DruidMetrics.GROUP_ID, new String[]{task.getGroupId()}
|
||||
),
|
||||
),
|
||||
task.getContextValue(DruidMetrics.TAGS)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -1,40 +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.index;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.IngestionSpec;
|
||||
import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
||||
|
||||
public class RealtimeAppenderatorIngestionSpec extends IngestionSpec<RealtimeIOConfig, RealtimeAppenderatorTuningConfig>
|
||||
{
|
||||
|
||||
@JsonCreator
|
||||
public RealtimeAppenderatorIngestionSpec(
|
||||
@JsonProperty("dataSchema") DataSchema dataSchema,
|
||||
@JsonProperty("ioConfig") RealtimeIOConfig ioConfig,
|
||||
@JsonProperty("tuningConfig") RealtimeAppenderatorTuningConfig tuningConfig
|
||||
)
|
||||
{
|
||||
super(dataSchema, ioConfig, tuningConfig);
|
||||
}
|
||||
}
|
|
@ -1,357 +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.index;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.incremental.AppendableIndexSpec;
|
||||
import org.apache.druid.segment.indexing.TuningConfig;
|
||||
import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
||||
import org.apache.druid.timeline.partition.ShardSpec;
|
||||
import org.joda.time.Period;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
|
||||
@JsonTypeName("realtime_appenderator")
|
||||
public class RealtimeAppenderatorTuningConfig implements AppenderatorConfig
|
||||
{
|
||||
private static final Period DEFAULT_INTERMEDIATE_PERSIST_PERIOD = new Period("PT10M");
|
||||
private static final int DEFAULT_MAX_PENDING_PERSISTS = 0;
|
||||
private static final ShardSpec DEFAULT_SHARD_SPEC = new NumberedShardSpec(0, 1);
|
||||
private static final IndexSpec DEFAULT_INDEX_SPEC = IndexSpec.DEFAULT;
|
||||
private static final Boolean DEFAULT_REPORT_PARSE_EXCEPTIONS = Boolean.FALSE;
|
||||
private static final long DEFAULT_HANDOFF_CONDITION_TIMEOUT = 0;
|
||||
private static final long DEFAULT_ALERT_TIMEOUT = 0;
|
||||
|
||||
private final AppendableIndexSpec appendableIndexSpec;
|
||||
private final int maxRowsInMemory;
|
||||
private final long maxBytesInMemory;
|
||||
private final boolean skipBytesInMemoryOverheadCheck;
|
||||
private final DynamicPartitionsSpec partitionsSpec;
|
||||
private final Period intermediatePersistPeriod;
|
||||
private final File basePersistDirectory;
|
||||
private final int maxPendingPersists;
|
||||
private final ShardSpec shardSpec;
|
||||
private final IndexSpec indexSpec;
|
||||
private final IndexSpec indexSpecForIntermediatePersists;
|
||||
private final boolean reportParseExceptions;
|
||||
private final long publishAndHandoffTimeout;
|
||||
private final long alertTimeout;
|
||||
@Nullable
|
||||
private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory;
|
||||
|
||||
private final boolean logParseExceptions;
|
||||
private final int maxParseExceptions;
|
||||
private final int maxSavedParseExceptions;
|
||||
|
||||
private final int numPersistThreads;
|
||||
|
||||
public RealtimeAppenderatorTuningConfig(
|
||||
@Nullable AppendableIndexSpec appendableIndexSpec,
|
||||
Integer maxRowsInMemory,
|
||||
@Nullable Long maxBytesInMemory,
|
||||
@Nullable Boolean skipBytesInMemoryOverheadCheck,
|
||||
@Nullable Integer maxRowsPerSegment,
|
||||
@Nullable Long maxTotalRows,
|
||||
Period intermediatePersistPeriod,
|
||||
File basePersistDirectory,
|
||||
Integer maxPendingPersists,
|
||||
ShardSpec shardSpec,
|
||||
IndexSpec indexSpec,
|
||||
@Nullable IndexSpec indexSpecForIntermediatePersists,
|
||||
Boolean reportParseExceptions,
|
||||
Long publishAndHandoffTimeout,
|
||||
Long alertTimeout,
|
||||
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
|
||||
@Nullable Boolean logParseExceptions,
|
||||
@Nullable Integer maxParseExceptions,
|
||||
@Nullable Integer maxSavedParseExceptions,
|
||||
@Nullable Integer numPersistThreads
|
||||
)
|
||||
{
|
||||
this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec;
|
||||
this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY_REALTIME : maxRowsInMemory;
|
||||
// initializing this to 0, it will be lazily initialized to a value
|
||||
// @see #getMaxBytesInMemoryOrDefault()
|
||||
this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory;
|
||||
this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck == null ?
|
||||
DEFAULT_SKIP_BYTES_IN_MEMORY_OVERHEAD_CHECK : skipBytesInMemoryOverheadCheck;
|
||||
this.partitionsSpec = new DynamicPartitionsSpec(maxRowsPerSegment, maxTotalRows);
|
||||
this.intermediatePersistPeriod = intermediatePersistPeriod == null
|
||||
? DEFAULT_INTERMEDIATE_PERSIST_PERIOD
|
||||
: intermediatePersistPeriod;
|
||||
this.basePersistDirectory = basePersistDirectory;
|
||||
this.maxPendingPersists = maxPendingPersists == null ? DEFAULT_MAX_PENDING_PERSISTS : maxPendingPersists;
|
||||
this.shardSpec = shardSpec == null ? DEFAULT_SHARD_SPEC : shardSpec;
|
||||
this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec;
|
||||
this.indexSpecForIntermediatePersists = indexSpecForIntermediatePersists == null ?
|
||||
this.indexSpec : indexSpecForIntermediatePersists;
|
||||
this.reportParseExceptions = reportParseExceptions == null
|
||||
? DEFAULT_REPORT_PARSE_EXCEPTIONS
|
||||
: reportParseExceptions;
|
||||
this.publishAndHandoffTimeout = publishAndHandoffTimeout == null
|
||||
? DEFAULT_HANDOFF_CONDITION_TIMEOUT
|
||||
: publishAndHandoffTimeout;
|
||||
Preconditions.checkArgument(this.publishAndHandoffTimeout >= 0, "publishAndHandoffTimeout must be >= 0");
|
||||
|
||||
this.alertTimeout = alertTimeout == null ? DEFAULT_ALERT_TIMEOUT : alertTimeout;
|
||||
Preconditions.checkArgument(this.alertTimeout >= 0, "alertTimeout must be >= 0");
|
||||
this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory;
|
||||
|
||||
if (this.reportParseExceptions) {
|
||||
this.maxParseExceptions = 0;
|
||||
this.maxSavedParseExceptions = maxSavedParseExceptions == null ? 0 : Math.min(1, maxSavedParseExceptions);
|
||||
} else {
|
||||
this.maxParseExceptions = maxParseExceptions == null
|
||||
? TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS
|
||||
: maxParseExceptions;
|
||||
this.maxSavedParseExceptions = maxSavedParseExceptions == null
|
||||
? TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS
|
||||
: maxSavedParseExceptions;
|
||||
}
|
||||
this.logParseExceptions = logParseExceptions == null
|
||||
? TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS
|
||||
: logParseExceptions;
|
||||
this.numPersistThreads = numPersistThreads == null ?
|
||||
DEFAULT_NUM_PERSIST_THREADS : Math.max(numPersistThreads, DEFAULT_NUM_PERSIST_THREADS);
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
private RealtimeAppenderatorTuningConfig(
|
||||
@JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec,
|
||||
@JsonProperty("maxRowsInMemory") Integer maxRowsInMemory,
|
||||
@JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory,
|
||||
@JsonProperty("skipBytesInMemoryOverheadCheck") @Nullable Boolean skipBytesInMemoryOverheadCheck,
|
||||
@JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment,
|
||||
@JsonProperty("maxTotalRows") @Nullable Long maxTotalRows,
|
||||
@JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod,
|
||||
@JsonProperty("maxPendingPersists") Integer maxPendingPersists,
|
||||
@JsonProperty("shardSpec") ShardSpec shardSpec,
|
||||
@JsonProperty("indexSpec") IndexSpec indexSpec,
|
||||
@JsonProperty("indexSpecForIntermediatePersists") @Nullable IndexSpec indexSpecForIntermediatePersists,
|
||||
@JsonProperty("reportParseExceptions") Boolean reportParseExceptions,
|
||||
@JsonProperty("publishAndHandoffTimeout") Long publishAndHandoffTimeout,
|
||||
@JsonProperty("alertTimeout") Long alertTimeout,
|
||||
@JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
|
||||
@JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions,
|
||||
@JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions,
|
||||
@JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions,
|
||||
@JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads
|
||||
)
|
||||
{
|
||||
this(
|
||||
appendableIndexSpec,
|
||||
maxRowsInMemory,
|
||||
maxBytesInMemory,
|
||||
skipBytesInMemoryOverheadCheck,
|
||||
maxRowsPerSegment,
|
||||
maxTotalRows,
|
||||
intermediatePersistPeriod,
|
||||
null,
|
||||
maxPendingPersists,
|
||||
shardSpec,
|
||||
indexSpec,
|
||||
indexSpecForIntermediatePersists,
|
||||
reportParseExceptions,
|
||||
publishAndHandoffTimeout,
|
||||
alertTimeout,
|
||||
segmentWriteOutMediumFactory,
|
||||
logParseExceptions,
|
||||
maxParseExceptions,
|
||||
maxSavedParseExceptions,
|
||||
numPersistThreads
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public AppendableIndexSpec getAppendableIndexSpec()
|
||||
{
|
||||
return appendableIndexSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public int getMaxRowsInMemory()
|
||||
{
|
||||
return maxRowsInMemory;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public long getMaxBytesInMemory()
|
||||
{
|
||||
return maxBytesInMemory;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public boolean isSkipBytesInMemoryOverheadCheck()
|
||||
{
|
||||
return skipBytesInMemoryOverheadCheck;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public Integer getMaxRowsPerSegment()
|
||||
{
|
||||
return partitionsSpec.getMaxRowsPerSegment();
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
@Nullable
|
||||
public Long getMaxTotalRows()
|
||||
{
|
||||
return partitionsSpec.getMaxTotalRows();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DynamicPartitionsSpec getPartitionsSpec()
|
||||
{
|
||||
return partitionsSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public Period getIntermediatePersistPeriod()
|
||||
{
|
||||
return intermediatePersistPeriod;
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getBasePersistDirectory()
|
||||
{
|
||||
return Preconditions.checkNotNull(basePersistDirectory, "basePersistDirectory not set");
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public int getMaxPendingPersists()
|
||||
{
|
||||
return maxPendingPersists;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public ShardSpec getShardSpec()
|
||||
{
|
||||
return shardSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public IndexSpec getIndexSpec()
|
||||
{
|
||||
return indexSpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public IndexSpec getIndexSpecForIntermediatePersists()
|
||||
{
|
||||
return indexSpecForIntermediatePersists;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public boolean isReportParseExceptions()
|
||||
{
|
||||
return reportParseExceptions;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getPublishAndHandoffTimeout()
|
||||
{
|
||||
return publishAndHandoffTimeout;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getAlertTimeout()
|
||||
{
|
||||
return alertTimeout;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
@Nullable
|
||||
public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory()
|
||||
{
|
||||
return segmentWriteOutMediumFactory;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isLogParseExceptions()
|
||||
{
|
||||
return logParseExceptions;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getMaxParseExceptions()
|
||||
{
|
||||
return maxParseExceptions;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getMaxSavedParseExceptions()
|
||||
{
|
||||
return maxSavedParseExceptions;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public int getNumPersistThreads()
|
||||
{
|
||||
return numPersistThreads;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RealtimeAppenderatorTuningConfig withBasePersistDirectory(File dir)
|
||||
{
|
||||
return new RealtimeAppenderatorTuningConfig(
|
||||
appendableIndexSpec,
|
||||
maxRowsInMemory,
|
||||
maxBytesInMemory,
|
||||
skipBytesInMemoryOverheadCheck,
|
||||
partitionsSpec.getMaxRowsPerSegment(),
|
||||
partitionsSpec.getMaxTotalRows(),
|
||||
intermediatePersistPeriod,
|
||||
dir,
|
||||
maxPendingPersists,
|
||||
shardSpec,
|
||||
indexSpec,
|
||||
indexSpecForIntermediatePersists,
|
||||
reportParseExceptions,
|
||||
publishAndHandoffTimeout,
|
||||
alertTimeout,
|
||||
segmentWriteOutMediumFactory,
|
||||
logParseExceptions,
|
||||
maxParseExceptions,
|
||||
maxSavedParseExceptions,
|
||||
numPersistThreads
|
||||
);
|
||||
}
|
||||
}
|
|
@ -28,54 +28,49 @@ import org.apache.druid.java.util.metrics.MonitorUtils;
|
|||
import org.apache.druid.query.DruidMetrics;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMetersTotals;
|
||||
import org.apache.druid.segment.realtime.FireDepartment;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||
import org.apache.druid.segment.realtime.SegmentGenerationMetrics;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Replaces the old RealtimeMetricsMonitor for indexing tasks that use a single FireDepartment, with changes to
|
||||
* read row ingestion stats from RowIngestionMeters (which supports moving averages) instead of FireDepartmentMetrics.
|
||||
* See comment on RowIngestionMeters for more information regarding relationship between RowIngestionMeters and
|
||||
* FireDepartmentMetrics.
|
||||
* Emits metrics from {@link SegmentGenerationMetrics} and {@link RowIngestionMeters}.
|
||||
*/
|
||||
public class TaskRealtimeMetricsMonitor extends AbstractMonitor
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(TaskRealtimeMetricsMonitor.class);
|
||||
|
||||
private final FireDepartment fireDepartment;
|
||||
private final SegmentGenerationMetrics segmentGenerationMetrics;
|
||||
private final RowIngestionMeters rowIngestionMeters;
|
||||
private final Map<String, String[]> dimensions;
|
||||
@Nullable
|
||||
private final Map<String, Object> metricTags;
|
||||
|
||||
private FireDepartmentMetrics previousFireDepartmentMetrics;
|
||||
private SegmentGenerationMetrics previousSegmentGenerationMetrics;
|
||||
private RowIngestionMetersTotals previousRowIngestionMetersTotals;
|
||||
|
||||
public TaskRealtimeMetricsMonitor(
|
||||
FireDepartment fireDepartment,
|
||||
SegmentGenerationMetrics segmentGenerationMetrics,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
Map<String, String[]> dimensions,
|
||||
@Nullable Map<String, Object> metricTags
|
||||
)
|
||||
{
|
||||
this.fireDepartment = fireDepartment;
|
||||
this.segmentGenerationMetrics = segmentGenerationMetrics;
|
||||
this.rowIngestionMeters = rowIngestionMeters;
|
||||
this.dimensions = ImmutableMap.copyOf(dimensions);
|
||||
this.metricTags = metricTags;
|
||||
previousFireDepartmentMetrics = new FireDepartmentMetrics();
|
||||
previousSegmentGenerationMetrics = new SegmentGenerationMetrics();
|
||||
previousRowIngestionMetersTotals = new RowIngestionMetersTotals(0, 0, 0, 0, 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean doMonitor(ServiceEmitter emitter)
|
||||
{
|
||||
FireDepartmentMetrics metrics = fireDepartment.getMetrics().snapshot();
|
||||
SegmentGenerationMetrics metrics = segmentGenerationMetrics.snapshot();
|
||||
RowIngestionMetersTotals rowIngestionMetersTotals = rowIngestionMeters.getTotals();
|
||||
|
||||
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder()
|
||||
.setDimension(DruidMetrics.DATASOURCE, fireDepartment.getDataSchema().getDataSource());
|
||||
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder();
|
||||
MonitorUtils.addDimensionsToBuilder(builder, dimensions);
|
||||
|
||||
final long thrownAway = rowIngestionMetersTotals.getThrownAway() - previousRowIngestionMetersTotals.getThrownAway();
|
||||
|
@ -103,7 +98,7 @@ public class TaskRealtimeMetricsMonitor extends AbstractMonitor
|
|||
|
||||
emitter.emit(builder.setMetric("ingest/events/processed", rowIngestionMetersTotals.getProcessed() - previousRowIngestionMetersTotals.getProcessed()));
|
||||
|
||||
final long dedup = metrics.dedup() - previousFireDepartmentMetrics.dedup();
|
||||
final long dedup = metrics.dedup() - previousSegmentGenerationMetrics.dedup();
|
||||
if (dedup > 0) {
|
||||
log.warn("[%,d] duplicate events!", dedup);
|
||||
}
|
||||
|
@ -115,21 +110,21 @@ public class TaskRealtimeMetricsMonitor extends AbstractMonitor
|
|||
)
|
||||
);
|
||||
|
||||
emitter.emit(builder.setMetric("ingest/rows/output", metrics.rowOutput() - previousFireDepartmentMetrics.rowOutput()));
|
||||
emitter.emit(builder.setMetric("ingest/persists/count", metrics.numPersists() - previousFireDepartmentMetrics.numPersists()));
|
||||
emitter.emit(builder.setMetric("ingest/persists/time", metrics.persistTimeMillis() - previousFireDepartmentMetrics.persistTimeMillis()));
|
||||
emitter.emit(builder.setMetric("ingest/persists/cpu", metrics.persistCpuTime() - previousFireDepartmentMetrics.persistCpuTime()));
|
||||
emitter.emit(builder.setMetric("ingest/rows/output", metrics.rowOutput() - previousSegmentGenerationMetrics.rowOutput()));
|
||||
emitter.emit(builder.setMetric("ingest/persists/count", metrics.numPersists() - previousSegmentGenerationMetrics.numPersists()));
|
||||
emitter.emit(builder.setMetric("ingest/persists/time", metrics.persistTimeMillis() - previousSegmentGenerationMetrics.persistTimeMillis()));
|
||||
emitter.emit(builder.setMetric("ingest/persists/cpu", metrics.persistCpuTime() - previousSegmentGenerationMetrics.persistCpuTime()));
|
||||
emitter.emit(
|
||||
builder.setMetric(
|
||||
"ingest/persists/backPressure",
|
||||
metrics.persistBackPressureMillis() - previousFireDepartmentMetrics.persistBackPressureMillis()
|
||||
metrics.persistBackPressureMillis() - previousSegmentGenerationMetrics.persistBackPressureMillis()
|
||||
)
|
||||
);
|
||||
emitter.emit(builder.setMetric("ingest/persists/failed", metrics.failedPersists() - previousFireDepartmentMetrics.failedPersists()));
|
||||
emitter.emit(builder.setMetric("ingest/handoff/failed", metrics.failedHandoffs() - previousFireDepartmentMetrics.failedHandoffs()));
|
||||
emitter.emit(builder.setMetric("ingest/merge/time", metrics.mergeTimeMillis() - previousFireDepartmentMetrics.mergeTimeMillis()));
|
||||
emitter.emit(builder.setMetric("ingest/merge/cpu", metrics.mergeCpuTime() - previousFireDepartmentMetrics.mergeCpuTime()));
|
||||
emitter.emit(builder.setMetric("ingest/handoff/count", metrics.handOffCount() - previousFireDepartmentMetrics.handOffCount()));
|
||||
emitter.emit(builder.setMetric("ingest/persists/failed", metrics.failedPersists() - previousSegmentGenerationMetrics.failedPersists()));
|
||||
emitter.emit(builder.setMetric("ingest/handoff/failed", metrics.failedHandoffs() - previousSegmentGenerationMetrics.failedHandoffs()));
|
||||
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()));
|
||||
|
||||
long messageGap = metrics.messageGap();
|
||||
|
@ -143,7 +138,7 @@ public class TaskRealtimeMetricsMonitor extends AbstractMonitor
|
|||
}
|
||||
|
||||
previousRowIngestionMetersTotals = rowIngestionMetersTotals;
|
||||
previousFireDepartmentMetrics = metrics;
|
||||
previousSegmentGenerationMetrics = metrics;
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,853 +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.task;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import org.apache.druid.data.input.Committer;
|
||||
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.discovery.DiscoveryDruidNode;
|
||||
import org.apache.druid.discovery.LookupNodeService;
|
||||
import org.apache.druid.discovery.NodeRole;
|
||||
import org.apache.druid.indexer.IngestionState;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
|
||||
import org.apache.druid.indexer.report.IngestionStatsAndErrors;
|
||||
import org.apache.druid.indexer.report.IngestionStatsAndErrorsTaskReport;
|
||||
import org.apache.druid.indexer.report.TaskContextReport;
|
||||
import org.apache.druid.indexer.report.TaskReport;
|
||||
import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator;
|
||||
import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker;
|
||||
import org.apache.druid.indexing.common.LockGranularity;
|
||||
import org.apache.druid.indexing.common.TaskLock;
|
||||
import org.apache.druid.indexing.common.TaskLockType;
|
||||
import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.common.actions.SegmentAllocateAction;
|
||||
import org.apache.druid.indexing.common.actions.SegmentLockAcquireAction;
|
||||
import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction;
|
||||
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||
import org.apache.druid.indexing.common.actions.TaskLocks;
|
||||
import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction;
|
||||
import org.apache.druid.indexing.common.config.TaskConfig;
|
||||
import org.apache.druid.indexing.common.index.RealtimeAppenderatorIngestionSpec;
|
||||
import org.apache.druid.indexing.common.index.RealtimeAppenderatorTuningConfig;
|
||||
import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.UOE;
|
||||
import org.apache.druid.java.util.common.parsers.ParseException;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
import org.apache.druid.query.NoopQueryRunner;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryRunner;
|
||||
import org.apache.druid.segment.SegmentUtils;
|
||||
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.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
||||
import org.apache.druid.segment.realtime.FireDepartment;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||
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.StreamAppenderatorDriver;
|
||||
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
|
||||
import org.apache.druid.segment.realtime.firehose.ChatHandler;
|
||||
import org.apache.druid.segment.realtime.firehose.ClippedFirehoseFactory;
|
||||
import org.apache.druid.segment.realtime.firehose.EventReceiverFirehoseFactory;
|
||||
import org.apache.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory;
|
||||
import org.apache.druid.segment.realtime.plumber.Committers;
|
||||
import org.apache.druid.server.security.Action;
|
||||
import org.apache.druid.server.security.AuthorizerMapper;
|
||||
import org.apache.druid.server.security.ResourceAction;
|
||||
import org.apache.druid.timeline.partition.NumberedPartialShardSpec;
|
||||
import org.apache.druid.utils.CloseableUtils;
|
||||
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import javax.ws.rs.GET;
|
||||
import javax.ws.rs.Path;
|
||||
import javax.ws.rs.Produces;
|
||||
import javax.ws.rs.core.Context;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Queue;
|
||||
import java.util.Set;
|
||||
import java.util.Timer;
|
||||
import java.util.TimerTask;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
@Deprecated
|
||||
public class AppenderatorDriverRealtimeIndexTask extends AbstractTask
|
||||
implements ChatHandler, PendingSegmentAllocatingTask
|
||||
{
|
||||
public static final String TYPE = "index_realtime_appenderator";
|
||||
private static final String CTX_KEY_LOOKUP_TIER = "lookupTier";
|
||||
|
||||
private static final EmittingLogger log = new EmittingLogger(RealtimeIndexTask.class);
|
||||
|
||||
private static String makeTaskId(RealtimeAppenderatorIngestionSpec spec)
|
||||
{
|
||||
return StringUtils.format(
|
||||
"index_realtime_%s_%d_%s_%s",
|
||||
spec.getDataSchema().getDataSource(),
|
||||
spec.getTuningConfig().getShardSpec().getPartitionNum(),
|
||||
DateTimes.nowUtc(),
|
||||
RealtimeIndexTask.makeRandomId()
|
||||
);
|
||||
}
|
||||
|
||||
@JsonIgnore
|
||||
private final RealtimeAppenderatorIngestionSpec spec;
|
||||
|
||||
@JsonIgnore
|
||||
private final Queue<ListenableFuture<SegmentsAndCommitMetadata>> pendingHandoffs;
|
||||
|
||||
@JsonIgnore
|
||||
private volatile Appenderator appenderator = null;
|
||||
|
||||
@JsonIgnore
|
||||
private volatile Firehose firehose = null;
|
||||
|
||||
@JsonIgnore
|
||||
private volatile FireDepartmentMetrics metrics = null;
|
||||
|
||||
@JsonIgnore
|
||||
private volatile boolean gracefullyStopped = false;
|
||||
|
||||
@JsonIgnore
|
||||
private volatile boolean finishingJob = false;
|
||||
|
||||
@JsonIgnore
|
||||
private volatile Thread runThread = null;
|
||||
|
||||
@JsonIgnore
|
||||
private final LockGranularity lockGranularity;
|
||||
|
||||
@JsonIgnore
|
||||
@MonotonicNonNull
|
||||
private ParseExceptionHandler parseExceptionHandler;
|
||||
|
||||
@JsonIgnore
|
||||
@MonotonicNonNull
|
||||
private IngestionState ingestionState;
|
||||
|
||||
@JsonIgnore
|
||||
@MonotonicNonNull
|
||||
private AuthorizerMapper authorizerMapper;
|
||||
|
||||
@JsonIgnore
|
||||
@MonotonicNonNull
|
||||
private RowIngestionMeters rowIngestionMeters;
|
||||
|
||||
@JsonIgnore
|
||||
@MonotonicNonNull
|
||||
private String errorMsg;
|
||||
|
||||
@JsonCreator
|
||||
public AppenderatorDriverRealtimeIndexTask(
|
||||
@JsonProperty("id") String id,
|
||||
@JsonProperty("resource") TaskResource taskResource,
|
||||
@JsonProperty("spec") RealtimeAppenderatorIngestionSpec spec,
|
||||
@JsonProperty("context") Map<String, Object> context
|
||||
)
|
||||
{
|
||||
super(
|
||||
id == null ? makeTaskId(spec) : id,
|
||||
StringUtils.format("index_realtime_appenderator_%s", spec.getDataSchema().getDataSource()),
|
||||
taskResource,
|
||||
spec.getDataSchema().getDataSource(),
|
||||
context
|
||||
);
|
||||
this.spec = spec;
|
||||
this.pendingHandoffs = new ConcurrentLinkedQueue<>();
|
||||
|
||||
this.ingestionState = IngestionState.NOT_STARTED;
|
||||
this.lockGranularity = getContextValue(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK)
|
||||
? LockGranularity.TIME_CHUNK
|
||||
: LockGranularity.SEGMENT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPriority()
|
||||
{
|
||||
return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_REALTIME_TASK_PRIORITY);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonIgnore
|
||||
@Nonnull
|
||||
public Set<ResourceAction> getInputSourceResources() throws UOE
|
||||
{
|
||||
throw new UOE(StringUtils.format(
|
||||
"Task type [%s], does not support input source based security",
|
||||
getType()
|
||||
));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getNodeType()
|
||||
{
|
||||
return "realtime";
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> QueryRunner<T> getQueryRunner(Query<T> query)
|
||||
{
|
||||
if (appenderator == null) {
|
||||
// Not yet initialized, no data yet, just return a noop runner.
|
||||
return new NoopQueryRunner<>();
|
||||
}
|
||||
|
||||
return (queryPlus, responseContext) -> queryPlus.run(appenderator, responseContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsQueries()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isReady(TaskActionClient taskActionClient)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTaskAllocatorId()
|
||||
{
|
||||
return getGroupId();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus runTask(final TaskToolbox toolbox)
|
||||
{
|
||||
runThread = Thread.currentThread();
|
||||
authorizerMapper = toolbox.getAuthorizerMapper();
|
||||
rowIngestionMeters = toolbox.getRowIngestionMetersFactory().createRowIngestionMeters();
|
||||
parseExceptionHandler = new ParseExceptionHandler(
|
||||
rowIngestionMeters,
|
||||
spec.getTuningConfig().isLogParseExceptions(),
|
||||
spec.getTuningConfig().getMaxParseExceptions(),
|
||||
spec.getTuningConfig().getMaxSavedParseExceptions()
|
||||
);
|
||||
|
||||
setupTimeoutAlert();
|
||||
|
||||
DataSchema dataSchema = spec.getDataSchema();
|
||||
RealtimeAppenderatorTuningConfig tuningConfig = spec.getTuningConfig()
|
||||
.withBasePersistDirectory(toolbox.getPersistDir());
|
||||
|
||||
final FireDepartment fireDepartmentForMetrics =
|
||||
new FireDepartment(dataSchema, new RealtimeIOConfig(null, null), null);
|
||||
|
||||
final TaskRealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build(
|
||||
this,
|
||||
fireDepartmentForMetrics,
|
||||
rowIngestionMeters
|
||||
);
|
||||
|
||||
this.metrics = fireDepartmentForMetrics.getMetrics();
|
||||
|
||||
final Supplier<Committer> committerSupplier = Committers.nilSupplier();
|
||||
|
||||
DiscoveryDruidNode discoveryDruidNode = createDiscoveryDruidNode(toolbox);
|
||||
|
||||
appenderator = newAppenderator(dataSchema, tuningConfig, metrics, toolbox);
|
||||
final TaskLockType lockType = TaskLocks.determineLockTypeForAppend(getContext());
|
||||
StreamAppenderatorDriver driver = newDriver(dataSchema, appenderator, toolbox, metrics, lockType);
|
||||
|
||||
try {
|
||||
log.debug("Found chat handler of class[%s]", toolbox.getChatHandlerProvider().getClass().getName());
|
||||
toolbox.getChatHandlerProvider().register(getId(), this, false);
|
||||
|
||||
if (toolbox.getAppenderatorsManager().shouldTaskMakeNodeAnnouncements()) {
|
||||
toolbox.getDataSegmentServerAnnouncer().announce();
|
||||
toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode);
|
||||
}
|
||||
|
||||
driver.startJob(
|
||||
segmentId -> {
|
||||
try {
|
||||
if (lockGranularity == LockGranularity.SEGMENT) {
|
||||
return toolbox.getTaskActionClient().submit(
|
||||
new SegmentLockAcquireAction(
|
||||
TaskLockType.EXCLUSIVE,
|
||||
segmentId.getInterval(),
|
||||
segmentId.getVersion(),
|
||||
segmentId.getShardSpec().getPartitionNum(),
|
||||
1000L
|
||||
)
|
||||
).isOk();
|
||||
} else {
|
||||
final TaskLock lock = toolbox.getTaskActionClient().submit(
|
||||
new TimeChunkLockAcquireAction(
|
||||
TaskLockType.EXCLUSIVE,
|
||||
segmentId.getInterval(),
|
||||
1000L
|
||||
)
|
||||
);
|
||||
if (lock == null) {
|
||||
return false;
|
||||
}
|
||||
lock.assertNotRevoked();
|
||||
return true;
|
||||
}
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
// Set up metrics emission
|
||||
toolbox.addMonitor(metricsMonitor);
|
||||
|
||||
// Delay firehose connection to avoid claiming input resources while the plumber is starting up.
|
||||
final FirehoseFactory firehoseFactory = spec.getIOConfig().getFirehoseFactory();
|
||||
final boolean firehoseDrainableByClosing = isFirehoseDrainableByClosing(firehoseFactory);
|
||||
|
||||
int sequenceNumber = 0;
|
||||
String sequenceName = makeSequenceName(getId(), sequenceNumber);
|
||||
|
||||
final TransactionalSegmentPublisher publisher = (mustBeNullOrEmptyOverwriteSegments, segments, commitMetadata, map) -> {
|
||||
if (mustBeNullOrEmptyOverwriteSegments != null && !mustBeNullOrEmptyOverwriteSegments.isEmpty()) {
|
||||
throw new ISE(
|
||||
"Stream ingestion task unexpectedly attempted to overwrite segments: %s",
|
||||
SegmentUtils.commaSeparatedIdentifiers(mustBeNullOrEmptyOverwriteSegments)
|
||||
);
|
||||
}
|
||||
final SegmentTransactionalInsertAction action = SegmentTransactionalInsertAction.appendAction(
|
||||
segments,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
return toolbox.getTaskActionClient().submit(action);
|
||||
};
|
||||
|
||||
// Skip connecting firehose if we've been stopped before we got started.
|
||||
synchronized (this) {
|
||||
if (!gracefullyStopped) {
|
||||
firehose = firehoseFactory.connect(
|
||||
Preconditions.checkNotNull(spec.getDataSchema().getParser(), "inputRowParser"),
|
||||
toolbox.getIndexingTmpDir()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
ingestionState = IngestionState.BUILD_SEGMENTS;
|
||||
|
||||
// Time to read data!
|
||||
while (!gracefullyStopped && firehoseDrainableByClosing && firehose.hasMore()) {
|
||||
try {
|
||||
InputRow inputRow = firehose.nextRow();
|
||||
|
||||
if (inputRow == null) {
|
||||
log.debug("Discarded null row, considering thrownAway.");
|
||||
rowIngestionMeters.incrementThrownAway();
|
||||
} else {
|
||||
AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName, committerSupplier);
|
||||
|
||||
if (addResult.isOk()) {
|
||||
final boolean isPushRequired = addResult.isPushRequired(
|
||||
tuningConfig.getPartitionsSpec().getMaxRowsPerSegment(),
|
||||
tuningConfig.getPartitionsSpec().getMaxTotalRowsOr(DynamicPartitionsSpec.DEFAULT_MAX_TOTAL_ROWS)
|
||||
);
|
||||
if (isPushRequired) {
|
||||
publishSegments(driver, publisher, committerSupplier, sequenceName);
|
||||
sequenceNumber++;
|
||||
sequenceName = makeSequenceName(getId(), sequenceNumber);
|
||||
}
|
||||
} else {
|
||||
// Failure to allocate segment puts determinism at risk, bail out to be safe.
|
||||
// May want configurable behavior here at some point.
|
||||
// If we allow continuing, then consider blacklisting the interval for a while to avoid constant checks.
|
||||
throw new ISE("Could not allocate segment for row with timestamp[%s]", inputRow.getTimestamp());
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (ParseException e) {
|
||||
handleParseException(e);
|
||||
}
|
||||
}
|
||||
|
||||
ingestionState = IngestionState.COMPLETED;
|
||||
|
||||
if (!gracefullyStopped) {
|
||||
synchronized (this) {
|
||||
if (gracefullyStopped) {
|
||||
// Someone called stopGracefully after we checked the flag. That's okay, just stop now.
|
||||
log.info("Gracefully stopping.");
|
||||
} else {
|
||||
finishingJob = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (finishingJob) {
|
||||
log.info("Finishing job...");
|
||||
// Publish any remaining segments
|
||||
publishSegments(driver, publisher, committerSupplier, sequenceName);
|
||||
|
||||
waitForSegmentPublishAndHandoff(tuningConfig.getPublishAndHandoffTimeout());
|
||||
}
|
||||
} else if (firehose != null) {
|
||||
log.info("Task was gracefully stopped, will persist data before exiting");
|
||||
|
||||
persistAndWait(driver, committerSupplier.get());
|
||||
}
|
||||
}
|
||||
catch (Throwable e) {
|
||||
log.makeAlert(e, "Exception aborted realtime processing[%s]", dataSchema.getDataSource())
|
||||
.emit();
|
||||
errorMsg = Throwables.getStackTraceAsString(e);
|
||||
toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports());
|
||||
return TaskStatus.failure(
|
||||
getId(),
|
||||
errorMsg
|
||||
);
|
||||
}
|
||||
finally {
|
||||
toolbox.getChatHandlerProvider().unregister(getId());
|
||||
|
||||
CloseableUtils.closeAndSuppressExceptions(firehose, e -> log.warn("Failed to close Firehose"));
|
||||
appenderator.close();
|
||||
CloseableUtils.closeAndSuppressExceptions(driver, e -> log.warn("Failed to close AppenderatorDriver"));
|
||||
|
||||
toolbox.removeMonitor(metricsMonitor);
|
||||
|
||||
if (toolbox.getAppenderatorsManager().shouldTaskMakeNodeAnnouncements()) {
|
||||
toolbox.getDataSegmentServerAnnouncer().unannounce();
|
||||
toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode);
|
||||
}
|
||||
}
|
||||
|
||||
log.info("Job done!");
|
||||
toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports());
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canRestore()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stopGracefully(TaskConfig taskConfig)
|
||||
{
|
||||
if (taskConfig.isRestoreTasksOnRestart()) {
|
||||
try {
|
||||
synchronized (this) {
|
||||
if (!gracefullyStopped) {
|
||||
gracefullyStopped = true;
|
||||
if (firehose == null) {
|
||||
log.info("stopGracefully: Firehose not started yet, so nothing to stop.");
|
||||
} else if (finishingJob) {
|
||||
log.info("stopGracefully: Interrupting finishJob.");
|
||||
runThread.interrupt();
|
||||
} else if (isFirehoseDrainableByClosing(spec.getIOConfig().getFirehoseFactory())) {
|
||||
log.info("stopGracefully: Draining firehose.");
|
||||
firehose.close();
|
||||
} else {
|
||||
log.info("stopGracefully: Cannot drain firehose by closing, interrupting run thread.");
|
||||
runThread.interrupt();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
} else {
|
||||
synchronized (this) {
|
||||
if (!gracefullyStopped) {
|
||||
// If task restore is not enabled, just interrupt immediately.
|
||||
gracefullyStopped = true;
|
||||
runThread.interrupt();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Public for tests.
|
||||
*/
|
||||
@JsonIgnore
|
||||
@VisibleForTesting
|
||||
public Firehose getFirehose()
|
||||
{
|
||||
return firehose;
|
||||
}
|
||||
|
||||
/**
|
||||
* Public for tests.
|
||||
*/
|
||||
@JsonIgnore
|
||||
@VisibleForTesting
|
||||
public FireDepartmentMetrics getMetrics()
|
||||
{
|
||||
return metrics;
|
||||
}
|
||||
|
||||
@JsonIgnore
|
||||
@VisibleForTesting
|
||||
public RowIngestionMeters getRowIngestionMeters()
|
||||
{
|
||||
return rowIngestionMeters;
|
||||
}
|
||||
|
||||
@JsonProperty("spec")
|
||||
public RealtimeAppenderatorIngestionSpec getSpec()
|
||||
{
|
||||
return spec;
|
||||
}
|
||||
|
||||
|
||||
@GET
|
||||
@Path("/rowStats")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
public Response getRowStats(
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
|
||||
Map<String, Object> returnMap = new HashMap<>();
|
||||
Map<String, Object> totalsMap = new HashMap<>();
|
||||
Map<String, Object> averagesMap = new HashMap<>();
|
||||
|
||||
totalsMap.put(
|
||||
RowIngestionMeters.BUILD_SEGMENTS,
|
||||
rowIngestionMeters.getTotals()
|
||||
);
|
||||
averagesMap.put(
|
||||
RowIngestionMeters.BUILD_SEGMENTS,
|
||||
rowIngestionMeters.getMovingAverages()
|
||||
);
|
||||
|
||||
returnMap.put("movingAverages", averagesMap);
|
||||
returnMap.put("totals", totalsMap);
|
||||
return Response.ok(returnMap).build();
|
||||
}
|
||||
|
||||
@GET
|
||||
@Path("/unparseableEvents")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
public Response getUnparseableEvents(
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
|
||||
List<ParseExceptionReport> events = IndexTaskUtils.getReportListFromSavedParseExceptions(
|
||||
parseExceptionHandler.getSavedParseExceptionReports()
|
||||
);
|
||||
return Response.ok(events).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Is a firehose from this factory drainable by closing it? If so, we should drain on stopGracefully rather than
|
||||
* abruptly stopping.
|
||||
* <p>
|
||||
* This is a hack to get around the fact that the Firehose and FirehoseFactory interfaces do not help us do this.
|
||||
* <p>
|
||||
* Protected for tests.
|
||||
*/
|
||||
protected boolean isFirehoseDrainableByClosing(FirehoseFactory firehoseFactory)
|
||||
{
|
||||
return firehoseFactory instanceof EventReceiverFirehoseFactory
|
||||
|| (firehoseFactory instanceof TimedShutoffFirehoseFactory
|
||||
&& isFirehoseDrainableByClosing(((TimedShutoffFirehoseFactory) firehoseFactory).getDelegateFactory()))
|
||||
|| (firehoseFactory instanceof ClippedFirehoseFactory
|
||||
&& isFirehoseDrainableByClosing(((ClippedFirehoseFactory) firehoseFactory).getDelegate()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a map of reports for the task.
|
||||
*
|
||||
* A successfull task should always have a null errorMsg. A falied task should always have a non-null
|
||||
* errorMsg.
|
||||
*
|
||||
* @return Map of reports for the task.
|
||||
*/
|
||||
private TaskReport.ReportMap getTaskCompletionReports()
|
||||
{
|
||||
return TaskReport.buildTaskReports(
|
||||
new IngestionStatsAndErrorsTaskReport(
|
||||
getId(),
|
||||
new IngestionStatsAndErrors(
|
||||
ingestionState,
|
||||
getTaskCompletionUnparseableEvents(),
|
||||
getTaskCompletionRowStats(),
|
||||
errorMsg,
|
||||
errorMsg == null,
|
||||
0L,
|
||||
Collections.emptyMap(),
|
||||
null,
|
||||
null
|
||||
)
|
||||
),
|
||||
new TaskContextReport(getId(), getContext())
|
||||
);
|
||||
}
|
||||
|
||||
private Map<String, Object> getTaskCompletionUnparseableEvents()
|
||||
{
|
||||
Map<String, Object> unparseableEventsMap = new HashMap<>();
|
||||
List<ParseExceptionReport> buildSegmentsParseExceptionMessages = IndexTaskUtils.getReportListFromSavedParseExceptions(
|
||||
parseExceptionHandler.getSavedParseExceptionReports()
|
||||
);
|
||||
if (buildSegmentsParseExceptionMessages != null) {
|
||||
unparseableEventsMap.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegmentsParseExceptionMessages);
|
||||
}
|
||||
return unparseableEventsMap;
|
||||
}
|
||||
|
||||
private Map<String, Object> getTaskCompletionRowStats()
|
||||
{
|
||||
Map<String, Object> metricsMap = new HashMap<>();
|
||||
metricsMap.put(
|
||||
RowIngestionMeters.BUILD_SEGMENTS,
|
||||
rowIngestionMeters.getTotals()
|
||||
);
|
||||
return metricsMap;
|
||||
}
|
||||
|
||||
private void handleParseException(ParseException pe)
|
||||
{
|
||||
parseExceptionHandler.handle(pe);
|
||||
|
||||
if (rowIngestionMeters.getUnparseable() + rowIngestionMeters.getProcessedWithError()
|
||||
> spec.getTuningConfig().getMaxParseExceptions()) {
|
||||
log.error("Max parse exceptions exceeded, terminating task...");
|
||||
throw new RuntimeException("Max parse exceptions exceeded, terminating task...");
|
||||
}
|
||||
}
|
||||
|
||||
private void setupTimeoutAlert()
|
||||
{
|
||||
if (spec.getTuningConfig().getAlertTimeout() > 0) {
|
||||
Timer timer = new Timer("RealtimeIndexTask-Timer", true);
|
||||
timer.schedule(
|
||||
new TimerTask()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
log.makeAlert(
|
||||
"RealtimeIndexTask for dataSource [%s] hasn't finished in configured time [%d] ms.",
|
||||
spec.getDataSchema().getDataSource(),
|
||||
spec.getTuningConfig().getAlertTimeout()
|
||||
).emit();
|
||||
}
|
||||
},
|
||||
spec.getTuningConfig().getAlertTimeout()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
private void publishSegments(
|
||||
StreamAppenderatorDriver driver,
|
||||
TransactionalSegmentPublisher publisher,
|
||||
Supplier<Committer> committerSupplier,
|
||||
String sequenceName
|
||||
)
|
||||
{
|
||||
final ListenableFuture<SegmentsAndCommitMetadata> publishFuture = driver.publish(
|
||||
publisher,
|
||||
committerSupplier.get(),
|
||||
Collections.singletonList(sequenceName)
|
||||
);
|
||||
pendingHandoffs.add(Futures.transformAsync(
|
||||
publishFuture,
|
||||
driver::registerHandoff,
|
||||
MoreExecutors.directExecutor()
|
||||
));
|
||||
}
|
||||
|
||||
private void waitForSegmentPublishAndHandoff(long timeout) throws InterruptedException, ExecutionException,
|
||||
TimeoutException
|
||||
{
|
||||
if (!pendingHandoffs.isEmpty()) {
|
||||
ListenableFuture<?> allHandoffs = Futures.allAsList(pendingHandoffs);
|
||||
log.info("Waiting for handoffs");
|
||||
|
||||
|
||||
if (timeout > 0) {
|
||||
allHandoffs.get(timeout, TimeUnit.MILLISECONDS);
|
||||
} else {
|
||||
allHandoffs.get();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void persistAndWait(StreamAppenderatorDriver driver, Committer committer)
|
||||
{
|
||||
try {
|
||||
final CountDownLatch persistLatch = new CountDownLatch(1);
|
||||
driver.persist(
|
||||
new Committer()
|
||||
{
|
||||
@Override
|
||||
public Object getMetadata()
|
||||
{
|
||||
return committer.getMetadata();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
try {
|
||||
committer.run();
|
||||
}
|
||||
finally {
|
||||
persistLatch.countDown();
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
persistLatch.await();
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
log.debug(e, "Interrupted while finishing the job");
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.makeAlert(e, "Failed to finish realtime task").emit();
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
private DiscoveryDruidNode createDiscoveryDruidNode(TaskToolbox toolbox)
|
||||
{
|
||||
LookupNodeService lookupNodeService = getContextValue(CTX_KEY_LOOKUP_TIER) == null ?
|
||||
toolbox.getLookupNodeService() :
|
||||
new LookupNodeService(getContextValue(CTX_KEY_LOOKUP_TIER));
|
||||
return new DiscoveryDruidNode(
|
||||
toolbox.getDruidNode(),
|
||||
NodeRole.PEON,
|
||||
ImmutableMap.of(
|
||||
toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(),
|
||||
lookupNodeService.getName(), lookupNodeService
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private Appenderator newAppenderator(
|
||||
final DataSchema dataSchema,
|
||||
final RealtimeAppenderatorTuningConfig tuningConfig,
|
||||
final FireDepartmentMetrics metrics,
|
||||
final TaskToolbox toolbox
|
||||
)
|
||||
{
|
||||
return toolbox.getAppenderatorsManager().createRealtimeAppenderatorForTask(
|
||||
null,
|
||||
getId(),
|
||||
dataSchema,
|
||||
tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()),
|
||||
metrics,
|
||||
toolbox.getSegmentPusher(),
|
||||
toolbox.getJsonMapper(),
|
||||
toolbox.getIndexIO(),
|
||||
toolbox.getIndexMergerV9(),
|
||||
toolbox.getQueryRunnerFactoryConglomerate(),
|
||||
toolbox.getSegmentAnnouncer(),
|
||||
toolbox.getEmitter(),
|
||||
toolbox.getQueryProcessingPool(),
|
||||
toolbox.getJoinableFactory(),
|
||||
toolbox.getCache(),
|
||||
toolbox.getCacheConfig(),
|
||||
toolbox.getCachePopulatorStats(),
|
||||
rowIngestionMeters,
|
||||
parseExceptionHandler,
|
||||
isUseMaxMemoryEstimates(),
|
||||
toolbox.getCentralizedTableSchemaConfig()
|
||||
);
|
||||
}
|
||||
|
||||
private static StreamAppenderatorDriver newDriver(
|
||||
final DataSchema dataSchema,
|
||||
final Appenderator appenderator,
|
||||
final TaskToolbox toolbox,
|
||||
final FireDepartmentMetrics metrics,
|
||||
final TaskLockType lockType
|
||||
)
|
||||
{
|
||||
return new StreamAppenderatorDriver(
|
||||
appenderator,
|
||||
new ActionBasedSegmentAllocator(
|
||||
toolbox.getTaskActionClient(),
|
||||
dataSchema,
|
||||
(schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction(
|
||||
schema.getDataSource(),
|
||||
row.getTimestamp(),
|
||||
schema.getGranularitySpec().getQueryGranularity(),
|
||||
schema.getGranularitySpec().getSegmentGranularity(),
|
||||
sequenceName,
|
||||
previousSegmentId,
|
||||
skipSegmentLineageCheck,
|
||||
NumberedPartialShardSpec.instance(),
|
||||
LockGranularity.TIME_CHUNK,
|
||||
lockType
|
||||
)
|
||||
),
|
||||
toolbox.getSegmentHandoffNotifierFactory(),
|
||||
new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()),
|
||||
toolbox.getDataSegmentKiller(),
|
||||
toolbox.getJsonMapper(),
|
||||
metrics
|
||||
);
|
||||
}
|
||||
|
||||
private static String makeSequenceName(String taskId, int sequenceNumber)
|
||||
{
|
||||
return taskId + "_" + sequenceNumber;
|
||||
}
|
||||
}
|
||||
|
|
@ -27,7 +27,7 @@ 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.loading.DataSegmentPusher;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||
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.AppenderatorsManager;
|
||||
|
@ -39,7 +39,7 @@ public final class BatchAppenderators
|
|||
public static Appenderator newAppenderator(
|
||||
String taskId,
|
||||
AppenderatorsManager appenderatorsManager,
|
||||
FireDepartmentMetrics metrics,
|
||||
SegmentGenerationMetrics metrics,
|
||||
TaskToolbox toolbox,
|
||||
DataSchema dataSchema,
|
||||
AppenderatorConfig appenderatorConfig,
|
||||
|
@ -65,7 +65,7 @@ public final class BatchAppenderators
|
|||
public static Appenderator newAppenderator(
|
||||
String taskId,
|
||||
AppenderatorsManager appenderatorsManager,
|
||||
FireDepartmentMetrics metrics,
|
||||
SegmentGenerationMetrics metrics,
|
||||
TaskToolbox toolbox,
|
||||
DataSchema dataSchema,
|
||||
AppenderatorConfig appenderatorConfig,
|
||||
|
|
|
@ -84,12 +84,10 @@ import org.apache.druid.segment.incremental.RowIngestionMeters;
|
|||
import org.apache.druid.segment.indexing.BatchIOConfig;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.IngestionSpec;
|
||||
import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
||||
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.FireDepartment;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||
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.BaseAppenderatorDriver;
|
||||
|
@ -832,15 +830,9 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler, Pe
|
|||
final PartitionAnalysis partitionAnalysis
|
||||
) throws IOException, InterruptedException
|
||||
{
|
||||
final FireDepartment fireDepartmentForMetrics =
|
||||
new FireDepartment(dataSchema, new RealtimeIOConfig(null, null), null);
|
||||
FireDepartmentMetrics buildSegmentsFireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
|
||||
|
||||
final TaskRealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build(
|
||||
this,
|
||||
fireDepartmentForMetrics,
|
||||
buildSegmentsMeters
|
||||
);
|
||||
final SegmentGenerationMetrics buildSegmentsSegmentGenerationMetrics = new SegmentGenerationMetrics();
|
||||
final TaskRealtimeMetricsMonitor metricsMonitor =
|
||||
TaskRealtimeMetricsMonitorBuilder.build(this, buildSegmentsSegmentGenerationMetrics, buildSegmentsMeters);
|
||||
toolbox.addMonitor(metricsMonitor);
|
||||
|
||||
final PartitionsSpec partitionsSpec = partitionAnalysis.getPartitionsSpec();
|
||||
|
@ -894,7 +886,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler, Pe
|
|||
final Appenderator appenderator = BatchAppenderators.newAppenderator(
|
||||
effectiveId,
|
||||
toolbox.getAppenderatorsManager(),
|
||||
buildSegmentsFireDepartmentMetrics,
|
||||
buildSegmentsSegmentGenerationMetrics,
|
||||
toolbox,
|
||||
dataSchema,
|
||||
tuningConfig,
|
||||
|
|
|
@ -1,643 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.indexing.common.task;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.data.input.Committer;
|
||||
import org.apache.druid.data.input.Firehose;
|
||||
import org.apache.druid.data.input.FirehoseFactory;
|
||||
import org.apache.druid.discovery.DiscoveryDruidNode;
|
||||
import org.apache.druid.discovery.LookupNodeService;
|
||||
import org.apache.druid.discovery.NodeRole;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.TaskLock;
|
||||
import org.apache.druid.indexing.common.TaskLockType;
|
||||
import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.common.actions.LockReleaseAction;
|
||||
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||
import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction;
|
||||
import org.apache.druid.indexing.common.config.TaskConfig;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.UOE;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
import org.apache.druid.query.NoopQueryRunner;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryRunner;
|
||||
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
||||
import org.apache.druid.segment.indexing.RealtimeTuningConfig;
|
||||
import org.apache.druid.segment.realtime.FireDepartment;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||
import org.apache.druid.segment.realtime.RealtimeMetricsMonitor;
|
||||
import org.apache.druid.segment.realtime.SegmentPublisher;
|
||||
import org.apache.druid.segment.realtime.appenderator.SegmentSchemas;
|
||||
import org.apache.druid.segment.realtime.firehose.ClippedFirehoseFactory;
|
||||
import org.apache.druid.segment.realtime.firehose.EventReceiverFirehoseFactory;
|
||||
import org.apache.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory;
|
||||
import org.apache.druid.segment.realtime.plumber.Committers;
|
||||
import org.apache.druid.segment.realtime.plumber.Plumber;
|
||||
import org.apache.druid.segment.realtime.plumber.PlumberSchool;
|
||||
import org.apache.druid.segment.realtime.plumber.Plumbers;
|
||||
import org.apache.druid.segment.realtime.plumber.RealtimePlumberSchool;
|
||||
import org.apache.druid.segment.realtime.plumber.VersioningPolicy;
|
||||
import org.apache.druid.server.coordination.DataSegmentAnnouncer;
|
||||
import org.apache.druid.server.security.ResourceAction;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.utils.CloseableUtils;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.Timer;
|
||||
import java.util.TimerTask;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
|
||||
@Deprecated
|
||||
public class RealtimeIndexTask extends AbstractTask
|
||||
{
|
||||
public static final String TYPE = "index_realtime";
|
||||
public static final String CTX_KEY_LOOKUP_TIER = "lookupTier";
|
||||
|
||||
private static final EmittingLogger log = new EmittingLogger(RealtimeIndexTask.class);
|
||||
|
||||
private static final int TASK_ID_BITS_PER_SYMBOL = 4;
|
||||
private static final int TASK_ID_SYMBOL_MASK = (1 << TASK_ID_BITS_PER_SYMBOL) - 1;
|
||||
private static final int TASK_ID_LENGTH = Integer.SIZE / TASK_ID_BITS_PER_SYMBOL;
|
||||
|
||||
public static String makeRandomId()
|
||||
{
|
||||
final StringBuilder suffix = new StringBuilder(TASK_ID_LENGTH);
|
||||
int randomBits = ThreadLocalRandom.current().nextInt();
|
||||
for (int i = 0; i < TASK_ID_LENGTH; i++) {
|
||||
suffix.append((char) ('a' + ((randomBits >>> (i * TASK_ID_BITS_PER_SYMBOL)) & TASK_ID_SYMBOL_MASK)));
|
||||
}
|
||||
return suffix.toString();
|
||||
}
|
||||
|
||||
private static String makeTaskId(FireDepartment fireDepartment)
|
||||
{
|
||||
return makeTaskId(
|
||||
fireDepartment.getDataSchema().getDataSource(),
|
||||
fireDepartment.getTuningConfig().getShardSpec().getPartitionNum(),
|
||||
DateTimes.nowUtc(),
|
||||
makeRandomId()
|
||||
);
|
||||
}
|
||||
|
||||
static String makeTaskId(String dataSource, int partitionNumber, DateTime timestamp, String suffix)
|
||||
{
|
||||
return StringUtils.format(
|
||||
"index_realtime_%s_%d_%s_%s",
|
||||
dataSource,
|
||||
partitionNumber,
|
||||
timestamp,
|
||||
suffix
|
||||
);
|
||||
}
|
||||
|
||||
private static String makeDatasource(FireDepartment fireDepartment)
|
||||
{
|
||||
return fireDepartment.getDataSchema().getDataSource();
|
||||
}
|
||||
|
||||
@JsonIgnore
|
||||
private final FireDepartment spec;
|
||||
|
||||
@JsonIgnore
|
||||
private volatile Plumber plumber = null;
|
||||
|
||||
@JsonIgnore
|
||||
private volatile Firehose firehose = null;
|
||||
|
||||
@JsonIgnore
|
||||
private volatile FireDepartmentMetrics metrics = null;
|
||||
|
||||
@JsonIgnore
|
||||
private volatile boolean gracefullyStopped = false;
|
||||
|
||||
@JsonIgnore
|
||||
private volatile boolean finishingJob = false;
|
||||
|
||||
@JsonIgnore
|
||||
private volatile Thread runThread = null;
|
||||
|
||||
@JsonIgnore
|
||||
private volatile QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate = null;
|
||||
|
||||
@JsonCreator
|
||||
public RealtimeIndexTask(
|
||||
@JsonProperty("id") String id,
|
||||
@JsonProperty("resource") TaskResource taskResource,
|
||||
@JsonProperty("spec") FireDepartment fireDepartment,
|
||||
@JsonProperty("context") Map<String, Object> context
|
||||
)
|
||||
{
|
||||
super(
|
||||
id == null ? makeTaskId(fireDepartment) : id,
|
||||
StringUtils.format("index_realtime_%s", makeDatasource(fireDepartment)),
|
||||
taskResource,
|
||||
makeDatasource(fireDepartment),
|
||||
context
|
||||
);
|
||||
this.spec = fireDepartment;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPriority()
|
||||
{
|
||||
return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_REALTIME_TASK_PRIORITY);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return TYPE;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonIgnore
|
||||
@Nonnull
|
||||
public Set<ResourceAction> getInputSourceResources() throws UOE
|
||||
{
|
||||
throw new UOE(StringUtils.format(
|
||||
"Task type [%s], does not support input source based security",
|
||||
getType()
|
||||
));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getNodeType()
|
||||
{
|
||||
return "realtime";
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> QueryRunner<T> getQueryRunner(Query<T> query)
|
||||
{
|
||||
if (plumber == null) {
|
||||
// Not yet initialized, no data yet, just return a noop runner.
|
||||
return new NoopQueryRunner<>();
|
||||
}
|
||||
|
||||
return plumber.getQueryRunner(query);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsQueries()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isReady(TaskActionClient taskActionClient)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus runTask(final TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
runThread = Thread.currentThread();
|
||||
|
||||
if (this.plumber != null) {
|
||||
throw new IllegalStateException("Plumber must be null");
|
||||
}
|
||||
|
||||
setupTimeoutAlert();
|
||||
|
||||
boolean normalExit = true;
|
||||
|
||||
// It would be nice to get the PlumberSchool in the constructor. Although that will need jackson injectables for
|
||||
// stuff like the ServerView, which seems kind of odd? Perhaps revisit this when Guice has been introduced.
|
||||
|
||||
final SegmentPublisher segmentPublisher = new TaskActionSegmentPublisher(toolbox);
|
||||
|
||||
// NOTE: We talk to the coordinator in various places in the plumber and we could be more robust to issues
|
||||
// with the coordinator. Right now, we'll block/throw in whatever thread triggered the coordinator behavior,
|
||||
// which will typically be either the main data processing loop or the persist thread.
|
||||
|
||||
// Wrap default DataSegmentAnnouncer such that we unlock intervals as we unannounce segments
|
||||
final long lockTimeoutMs = getContextValue(Tasks.LOCK_TIMEOUT_KEY, Tasks.DEFAULT_LOCK_TIMEOUT_MILLIS);
|
||||
// Note: if lockTimeoutMs is larger than ServerConfig.maxIdleTime, http timeout error can occur while waiting for a
|
||||
// lock to be acquired.
|
||||
final DataSegmentAnnouncer lockingSegmentAnnouncer = new DataSegmentAnnouncer()
|
||||
{
|
||||
@Override
|
||||
public void announceSegment(final DataSegment segment) throws IOException
|
||||
{
|
||||
// Side effect: Calling announceSegment causes a lock to be acquired
|
||||
final TaskLock lock = Preconditions.checkNotNull(
|
||||
toolbox.getTaskActionClient().submit(
|
||||
new TimeChunkLockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs)
|
||||
),
|
||||
"Cannot acquire a lock for interval[%s]",
|
||||
segment.getInterval()
|
||||
);
|
||||
lock.assertNotRevoked();
|
||||
toolbox.getSegmentAnnouncer().announceSegment(segment);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void unannounceSegment(final DataSegment segment) throws IOException
|
||||
{
|
||||
try {
|
||||
toolbox.getSegmentAnnouncer().unannounceSegment(segment);
|
||||
}
|
||||
finally {
|
||||
toolbox.getTaskActionClient().submit(new LockReleaseAction(segment.getInterval()));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void announceSegments(Iterable<DataSegment> segments) throws IOException
|
||||
{
|
||||
// Side effect: Calling announceSegments causes locks to be acquired
|
||||
for (DataSegment segment : segments) {
|
||||
final TaskLock lock = Preconditions.checkNotNull(
|
||||
toolbox.getTaskActionClient().submit(
|
||||
new TimeChunkLockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs)
|
||||
),
|
||||
"Cannot acquire a lock for interval[%s]",
|
||||
segment.getInterval()
|
||||
);
|
||||
lock.assertNotRevoked();
|
||||
}
|
||||
toolbox.getSegmentAnnouncer().announceSegments(segments);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void unannounceSegments(Iterable<DataSegment> segments) throws IOException
|
||||
{
|
||||
try {
|
||||
toolbox.getSegmentAnnouncer().unannounceSegments(segments);
|
||||
}
|
||||
finally {
|
||||
for (DataSegment segment : segments) {
|
||||
toolbox.getTaskActionClient().submit(new LockReleaseAction(segment.getInterval()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void announceSegmentSchemas(String taskId, SegmentSchemas sinksSchema, SegmentSchemas sinksSchemaChange)
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeSegmentSchemasForTask(String taskId)
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
// NOTE: getVersion will block if there is lock contention, which will block plumber.getSink
|
||||
// NOTE: (and thus the firehose)
|
||||
|
||||
// Shouldn't usually happen, since we don't expect people to submit tasks that intersect with the
|
||||
// realtime window, but if they do it can be problematic. If we decide to care, we can use more threads in
|
||||
// the plumber such that waiting for the coordinator doesn't block data processing.
|
||||
final VersioningPolicy versioningPolicy = new VersioningPolicy()
|
||||
{
|
||||
@Override
|
||||
public String getVersion(final Interval interval)
|
||||
{
|
||||
try {
|
||||
// Side effect: Calling getVersion causes a lock to be acquired
|
||||
final TimeChunkLockAcquireAction action = new TimeChunkLockAcquireAction(
|
||||
TaskLockType.EXCLUSIVE,
|
||||
interval,
|
||||
lockTimeoutMs
|
||||
);
|
||||
final TaskLock lock = Preconditions.checkNotNull(
|
||||
toolbox.getTaskActionClient().submit(action),
|
||||
"Cannot acquire a lock for interval[%s]",
|
||||
interval
|
||||
);
|
||||
lock.assertNotRevoked();
|
||||
return lock.getVersion();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
DataSchema dataSchema = spec.getDataSchema();
|
||||
RealtimeIOConfig realtimeIOConfig = spec.getIOConfig();
|
||||
RealtimeTuningConfig tuningConfig = spec.getTuningConfig()
|
||||
.withBasePersistDirectory(toolbox.getPersistDir())
|
||||
.withVersioningPolicy(versioningPolicy);
|
||||
|
||||
final FireDepartment fireDepartment = new FireDepartment(
|
||||
dataSchema,
|
||||
realtimeIOConfig,
|
||||
tuningConfig
|
||||
);
|
||||
this.metrics = fireDepartment.getMetrics();
|
||||
final RealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build(this, fireDepartment);
|
||||
|
||||
this.queryRunnerFactoryConglomerate = toolbox.getQueryRunnerFactoryConglomerate();
|
||||
|
||||
// NOTE: This pusher selects path based purely on global configuration and the DataSegment, which means
|
||||
// NOTE: that redundant realtime tasks will upload to the same location. This can cause index.zip
|
||||
// NOTE: (partitionNum_index.zip for HDFS data storage) to mismatch, or it can cause historical nodes to load
|
||||
// NOTE: different instances of the "same" segment.
|
||||
final PlumberSchool plumberSchool = new RealtimePlumberSchool(
|
||||
toolbox.getEmitter(),
|
||||
toolbox.getQueryRunnerFactoryConglomerate(),
|
||||
toolbox.getSegmentPusher(),
|
||||
lockingSegmentAnnouncer,
|
||||
segmentPublisher,
|
||||
toolbox.getSegmentHandoffNotifierFactory(),
|
||||
toolbox.getQueryProcessingPool(),
|
||||
toolbox.getJoinableFactory(),
|
||||
toolbox.getIndexMergerV9(),
|
||||
toolbox.getIndexIO(),
|
||||
toolbox.getCache(),
|
||||
toolbox.getCacheConfig(),
|
||||
toolbox.getCachePopulatorStats(),
|
||||
toolbox.getJsonMapper()
|
||||
);
|
||||
|
||||
this.plumber = plumberSchool.findPlumber(dataSchema, tuningConfig, metrics);
|
||||
|
||||
final Supplier<Committer> committerSupplier = Committers.nilSupplier();
|
||||
|
||||
LookupNodeService lookupNodeService = getContextValue(CTX_KEY_LOOKUP_TIER) == null ?
|
||||
toolbox.getLookupNodeService() :
|
||||
new LookupNodeService((String) getContextValue(CTX_KEY_LOOKUP_TIER));
|
||||
DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(
|
||||
toolbox.getDruidNode(),
|
||||
NodeRole.PEON,
|
||||
ImmutableMap.of(
|
||||
toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(),
|
||||
lookupNodeService.getName(), lookupNodeService
|
||||
)
|
||||
);
|
||||
|
||||
try {
|
||||
toolbox.getDataSegmentServerAnnouncer().announce();
|
||||
toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode);
|
||||
|
||||
|
||||
plumber.startJob();
|
||||
|
||||
// Set up metrics emission
|
||||
toolbox.addMonitor(metricsMonitor);
|
||||
|
||||
// Delay firehose connection to avoid claiming input resources while the plumber is starting up.
|
||||
final FirehoseFactory firehoseFactory = spec.getIOConfig().getFirehoseFactory();
|
||||
final boolean firehoseDrainableByClosing = isFirehoseDrainableByClosing(firehoseFactory);
|
||||
|
||||
// Skip connecting firehose if we've been stopped before we got started.
|
||||
synchronized (this) {
|
||||
if (!gracefullyStopped) {
|
||||
firehose = firehoseFactory.connect(
|
||||
Preconditions.checkNotNull(spec.getDataSchema().getParser(), "inputRowParser"),
|
||||
toolbox.getIndexingTmpDir()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
// Time to read data!
|
||||
while (firehose != null && (!gracefullyStopped || firehoseDrainableByClosing) && firehose.hasMore()) {
|
||||
Plumbers.addNextRow(
|
||||
committerSupplier,
|
||||
firehose,
|
||||
plumber,
|
||||
tuningConfig.isReportParseExceptions(),
|
||||
metrics
|
||||
);
|
||||
}
|
||||
}
|
||||
catch (Throwable e) {
|
||||
normalExit = false;
|
||||
log.makeAlert(e, "Exception aborted realtime processing[%s]", dataSchema.getDataSource())
|
||||
.emit();
|
||||
throw e;
|
||||
}
|
||||
finally {
|
||||
if (normalExit) {
|
||||
try {
|
||||
// Persist if we had actually started.
|
||||
if (firehose != null) {
|
||||
log.info("Persisting remaining data.");
|
||||
|
||||
final Committer committer = committerSupplier.get();
|
||||
final CountDownLatch persistLatch = new CountDownLatch(1);
|
||||
plumber.persist(
|
||||
new Committer()
|
||||
{
|
||||
@Override
|
||||
public Object getMetadata()
|
||||
{
|
||||
return committer.getMetadata();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
try {
|
||||
committer.run();
|
||||
}
|
||||
finally {
|
||||
persistLatch.countDown();
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
persistLatch.await();
|
||||
}
|
||||
|
||||
if (gracefullyStopped) {
|
||||
log.info("Gracefully stopping.");
|
||||
} else {
|
||||
log.info("Finishing the job.");
|
||||
synchronized (this) {
|
||||
if (gracefullyStopped) {
|
||||
// Someone called stopGracefully after we checked the flag. That's okay, just stop now.
|
||||
log.info("Gracefully stopping.");
|
||||
} else {
|
||||
finishingJob = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (finishingJob) {
|
||||
plumber.finishJob();
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
log.debug(e, "Interrupted while finishing the job");
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.makeAlert(e, "Failed to finish realtime task").emit();
|
||||
throw e;
|
||||
}
|
||||
finally {
|
||||
if (firehose != null) {
|
||||
CloseableUtils.closeAndSuppressExceptions(firehose, e -> log.warn("Failed to close Firehose"));
|
||||
}
|
||||
toolbox.removeMonitor(metricsMonitor);
|
||||
}
|
||||
}
|
||||
|
||||
toolbox.getDataSegmentServerAnnouncer().unannounce();
|
||||
toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode);
|
||||
}
|
||||
|
||||
log.info("Job done!");
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canRestore()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stopGracefully(TaskConfig taskConfig)
|
||||
{
|
||||
if (taskConfig.isRestoreTasksOnRestart()) {
|
||||
try {
|
||||
synchronized (this) {
|
||||
if (!gracefullyStopped) {
|
||||
gracefullyStopped = true;
|
||||
if (firehose == null) {
|
||||
log.info("stopGracefully: Firehose not started yet, so nothing to stop.");
|
||||
} else if (finishingJob) {
|
||||
log.info("stopGracefully: Interrupting finishJob.");
|
||||
runThread.interrupt();
|
||||
} else if (isFirehoseDrainableByClosing(spec.getIOConfig().getFirehoseFactory())) {
|
||||
log.info("stopGracefully: Draining firehose.");
|
||||
firehose.close();
|
||||
} else {
|
||||
log.info("stopGracefully: Cannot drain firehose by closing, interrupting run thread.");
|
||||
runThread.interrupt();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
} else {
|
||||
synchronized (this) {
|
||||
if (!gracefullyStopped) {
|
||||
// If task restore is not enabled, just interrupt immediately.
|
||||
gracefullyStopped = true;
|
||||
runThread.interrupt();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Public for tests.
|
||||
*/
|
||||
@JsonIgnore
|
||||
public Firehose getFirehose()
|
||||
{
|
||||
return firehose;
|
||||
}
|
||||
|
||||
/**
|
||||
* Public for tests.
|
||||
*/
|
||||
@JsonIgnore
|
||||
public FireDepartmentMetrics getMetrics()
|
||||
{
|
||||
return metrics;
|
||||
}
|
||||
|
||||
@JsonProperty("spec")
|
||||
public FireDepartment getRealtimeIngestionSchema()
|
||||
{
|
||||
return spec;
|
||||
}
|
||||
|
||||
/**
|
||||
* Is a firehose from this factory drainable by closing it? If so, we should drain on stopGracefully rather than
|
||||
* abruptly stopping.
|
||||
*
|
||||
* This is a hack to get around the fact that the Firehose and FirehoseFactory interfaces do not help us do this.
|
||||
*
|
||||
* Protected for tests.
|
||||
*/
|
||||
protected boolean isFirehoseDrainableByClosing(FirehoseFactory firehoseFactory)
|
||||
{
|
||||
return firehoseFactory instanceof EventReceiverFirehoseFactory
|
||||
|| (firehoseFactory instanceof TimedShutoffFirehoseFactory
|
||||
&& isFirehoseDrainableByClosing(((TimedShutoffFirehoseFactory) firehoseFactory).getDelegateFactory()))
|
||||
|| (firehoseFactory instanceof ClippedFirehoseFactory
|
||||
&& isFirehoseDrainableByClosing(((ClippedFirehoseFactory) firehoseFactory).getDelegate()));
|
||||
}
|
||||
|
||||
public static class TaskActionSegmentPublisher implements SegmentPublisher
|
||||
{
|
||||
final TaskToolbox taskToolbox;
|
||||
|
||||
public TaskActionSegmentPublisher(TaskToolbox taskToolbox)
|
||||
{
|
||||
this.taskToolbox = taskToolbox;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void publishSegment(DataSegment segment) throws IOException
|
||||
{
|
||||
taskToolbox.publishSegments(ImmutableList.of(segment));
|
||||
}
|
||||
}
|
||||
|
||||
private void setupTimeoutAlert()
|
||||
{
|
||||
if (spec.getTuningConfig().getAlertTimeout() > 0) {
|
||||
Timer timer = new Timer("RealtimeIndexTask-Timer", true);
|
||||
timer.schedule(
|
||||
new TimerTask()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
log.makeAlert(
|
||||
"RealtimeIndexTask for dataSource [%s] hasn't finished in configured time [%d] ms.",
|
||||
spec.getDataSchema().getDataSource(),
|
||||
spec.getTuningConfig().getAlertTimeout()
|
||||
).emit();
|
||||
}
|
||||
},
|
||||
spec.getTuningConfig().getAlertTimeout()
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -81,8 +81,6 @@ import java.util.Set;
|
|||
@Type(name = PartialDimensionDistributionTask.TYPE, value = PartialDimensionDistributionTask.class),
|
||||
@Type(name = PartialGenericSegmentMergeTask.TYPE, value = PartialGenericSegmentMergeTask.class),
|
||||
@Type(name = HadoopIndexTask.TYPE, value = HadoopIndexTask.class),
|
||||
@Type(name = RealtimeIndexTask.TYPE, value = RealtimeIndexTask.class),
|
||||
@Type(name = AppenderatorDriverRealtimeIndexTask.TYPE, value = AppenderatorDriverRealtimeIndexTask.class),
|
||||
@Type(name = NoopTask.TYPE, value = NoopTask.class),
|
||||
@Type(name = CompactionTask.TYPE, value = CompactionTask.class)
|
||||
})
|
||||
|
|
|
@ -46,9 +46,7 @@ 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.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
||||
import org.apache.druid.segment.realtime.FireDepartment;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||
import org.apache.druid.segment.realtime.SegmentGenerationMetrics;
|
||||
import org.apache.druid.segment.realtime.appenderator.Appenderator;
|
||||
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
|
||||
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
|
||||
|
@ -171,19 +169,10 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
|
|||
) throws IOException, InterruptedException, ExecutionException, TimeoutException
|
||||
{
|
||||
final DataSchema dataSchema = ingestionSchema.getDataSchema();
|
||||
final FireDepartment fireDepartmentForMetrics = new FireDepartment(
|
||||
dataSchema,
|
||||
new RealtimeIOConfig(null, null),
|
||||
null
|
||||
);
|
||||
final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
|
||||
final SegmentGenerationMetrics segmentGenerationMetrics = new SegmentGenerationMetrics();
|
||||
buildSegmentsMeters = toolbox.getRowIngestionMetersFactory().createRowIngestionMeters();
|
||||
|
||||
TaskRealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build(
|
||||
this,
|
||||
fireDepartmentForMetrics,
|
||||
buildSegmentsMeters
|
||||
);
|
||||
final TaskRealtimeMetricsMonitor metricsMonitor =
|
||||
TaskRealtimeMetricsMonitorBuilder.build(this, segmentGenerationMetrics, buildSegmentsMeters);
|
||||
toolbox.addMonitor(metricsMonitor);
|
||||
|
||||
final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
|
||||
|
@ -206,7 +195,7 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
|
|||
final Appenderator appenderator = BatchAppenderators.newAppenderator(
|
||||
getId(),
|
||||
toolbox.getAppenderatorsManager(),
|
||||
fireDepartmentMetrics,
|
||||
segmentGenerationMetrics,
|
||||
toolbox,
|
||||
dataSchema,
|
||||
tuningConfig,
|
||||
|
|
|
@ -57,12 +57,10 @@ 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.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
||||
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.FireDepartment;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||
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;
|
||||
|
@ -368,15 +366,9 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand
|
|||
{
|
||||
final DataSchema dataSchema = ingestionSchema.getDataSchema();
|
||||
final GranularitySpec granularitySpec = dataSchema.getGranularitySpec();
|
||||
final FireDepartment fireDepartmentForMetrics =
|
||||
new FireDepartment(dataSchema, new RealtimeIOConfig(null, null), null);
|
||||
final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
|
||||
|
||||
TaskRealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build(
|
||||
this,
|
||||
fireDepartmentForMetrics,
|
||||
rowIngestionMeters
|
||||
);
|
||||
final SegmentGenerationMetrics segmentGenerationMetrics = new SegmentGenerationMetrics();
|
||||
final TaskRealtimeMetricsMonitor metricsMonitor =
|
||||
TaskRealtimeMetricsMonitorBuilder.build(this, segmentGenerationMetrics, rowIngestionMeters);
|
||||
toolbox.addMonitor(metricsMonitor);
|
||||
|
||||
final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
|
||||
|
@ -409,7 +401,7 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand
|
|||
final Appenderator appenderator = BatchAppenderators.newAppenderator(
|
||||
getId(),
|
||||
toolbox.getAppenderatorsManager(),
|
||||
fireDepartmentMetrics,
|
||||
segmentGenerationMetrics,
|
||||
toolbox,
|
||||
dataSchema,
|
||||
tuningConfig,
|
||||
|
@ -461,8 +453,6 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand
|
|||
} else {
|
||||
throw new ISE("Failed to add a row with timestamp[%s]", inputRow.getTimestamp());
|
||||
}
|
||||
|
||||
fireDepartmentMetrics.incrementProcessed();
|
||||
}
|
||||
|
||||
final SegmentsAndCommitMetadata pushed = driver.pushAllAndClear(pushTimeout);
|
||||
|
|
|
@ -38,7 +38,6 @@ import org.apache.druid.query.QueryContexts;
|
|||
import org.apache.druid.segment.incremental.ParseExceptionReport;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
|
|
@ -49,7 +49,7 @@ import org.apache.druid.query.QueryRunner;
|
|||
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.FireDepartmentMetrics;
|
||||
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;
|
||||
|
@ -182,7 +182,7 @@ public abstract class SeekableStreamIndexTask<PartitionIdType, SequenceOffsetTyp
|
|||
|
||||
public Appenderator newAppenderator(
|
||||
TaskToolbox toolbox,
|
||||
FireDepartmentMetrics metrics,
|
||||
SegmentGenerationMetrics metrics,
|
||||
RowIngestionMeters rowIngestionMeters,
|
||||
ParseExceptionHandler parseExceptionHandler
|
||||
)
|
||||
|
@ -215,7 +215,7 @@ public abstract class SeekableStreamIndexTask<PartitionIdType, SequenceOffsetTyp
|
|||
public StreamAppenderatorDriver newDriver(
|
||||
final Appenderator appenderator,
|
||||
final TaskToolbox toolbox,
|
||||
final FireDepartmentMetrics metrics
|
||||
final SegmentGenerationMetrics metrics
|
||||
)
|
||||
{
|
||||
return new StreamAppenderatorDriver(
|
||||
|
|
|
@ -67,7 +67,6 @@ import org.apache.druid.indexing.common.actions.TaskLocks;
|
|||
import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction;
|
||||
import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor;
|
||||
import org.apache.druid.indexing.common.task.IndexTaskUtils;
|
||||
import org.apache.druid.indexing.common.task.RealtimeIndexTask;
|
||||
import org.apache.druid.indexing.input.InputRowSchemas;
|
||||
import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
|
||||
import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
|
||||
|
@ -82,9 +81,7 @@ import org.apache.druid.metadata.PendingSegmentRecord;
|
|||
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.indexing.RealtimeIOConfig;
|
||||
import org.apache.druid.segment.realtime.FireDepartment;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||
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;
|
||||
|
@ -147,6 +144,8 @@ import java.util.stream.Collectors;
|
|||
@SuppressWarnings("CheckReturnValue")
|
||||
public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOffsetType, RecordType extends ByteEntity> implements ChatHandler
|
||||
{
|
||||
private static final String CTX_KEY_LOOKUP_TIER = "lookupTier";
|
||||
|
||||
public enum Status
|
||||
{
|
||||
NOT_STARTED,
|
||||
|
@ -226,7 +225,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
@MonotonicNonNull
|
||||
private ParseExceptionHandler parseExceptionHandler;
|
||||
@MonotonicNonNull
|
||||
private FireDepartmentMetrics fireDepartmentMetrics;
|
||||
private SegmentGenerationMetrics segmentGenerationMetrics;
|
||||
|
||||
@MonotonicNonNull
|
||||
private AuthorizerMapper authorizerMapper;
|
||||
|
@ -401,17 +400,13 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
|
||||
runThread = Thread.currentThread();
|
||||
|
||||
// Set up FireDepartmentMetrics
|
||||
final FireDepartment fireDepartmentForMetrics = new FireDepartment(
|
||||
task.getDataSchema(),
|
||||
new RealtimeIOConfig(null, null),
|
||||
null
|
||||
);
|
||||
this.fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
|
||||
TaskRealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build(task, fireDepartmentForMetrics, rowIngestionMeters);
|
||||
// Set up SegmentGenerationMetrics
|
||||
this.segmentGenerationMetrics = new SegmentGenerationMetrics();
|
||||
final TaskRealtimeMetricsMonitor metricsMonitor =
|
||||
TaskRealtimeMetricsMonitorBuilder.build(task, segmentGenerationMetrics, rowIngestionMeters);
|
||||
toolbox.addMonitor(metricsMonitor);
|
||||
|
||||
final String lookupTier = task.getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER);
|
||||
final String lookupTier = task.getContextValue(CTX_KEY_LOOKUP_TIER);
|
||||
final LookupNodeService lookupNodeService = lookupTier == null ?
|
||||
toolbox.getLookupNodeService() :
|
||||
new LookupNodeService(lookupTier);
|
||||
|
@ -436,8 +431,8 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
toolbox.getDataSegmentServerAnnouncer().announce();
|
||||
toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode);
|
||||
}
|
||||
appenderator = task.newAppenderator(toolbox, fireDepartmentMetrics, rowIngestionMeters, parseExceptionHandler);
|
||||
driver = task.newDriver(appenderator, toolbox, fireDepartmentMetrics);
|
||||
appenderator = task.newAppenderator(toolbox, segmentGenerationMetrics, rowIngestionMeters, parseExceptionHandler);
|
||||
driver = task.newDriver(appenderator, toolbox, segmentGenerationMetrics);
|
||||
|
||||
// Start up, set up initial sequences.
|
||||
final Object restoredMetadata = driver.startJob(
|
||||
|
@ -739,7 +734,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
if (!stillReading) {
|
||||
// We let the fireDepartmentMetrics know that all messages have been read. This way, some metrics such as
|
||||
// high message gap need not be reported
|
||||
fireDepartmentMetrics.markProcessingDone();
|
||||
segmentGenerationMetrics.markProcessingDone();
|
||||
}
|
||||
|
||||
if (System.currentTimeMillis() > nextCheckpointTime) {
|
||||
|
@ -784,7 +779,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
finally {
|
||||
try {
|
||||
// To handle cases where tasks stop reading due to stop request or exceptions
|
||||
fireDepartmentMetrics.markProcessingDone();
|
||||
segmentGenerationMetrics.markProcessingDone();
|
||||
driver.persist(committerSupplier.get()); // persist pending data
|
||||
}
|
||||
catch (Exception e) {
|
||||
|
@ -1442,9 +1437,9 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public FireDepartmentMetrics getFireDepartmentMetrics()
|
||||
public SegmentGenerationMetrics getSegmentGenerationMetrics()
|
||||
{
|
||||
return fireDepartmentMetrics;
|
||||
return segmentGenerationMetrics;
|
||||
}
|
||||
|
||||
public void stopForcefully()
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.incremental.AppendableIndexSpec;
|
||||
import org.apache.druid.segment.indexing.RealtimeTuningConfig;
|
||||
import org.apache.druid.segment.indexing.TuningConfig;
|
||||
import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
|
||||
|
@ -31,12 +30,17 @@ import org.joda.time.Period;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.time.Duration;
|
||||
import java.util.Objects;
|
||||
|
||||
public abstract class SeekableStreamIndexTaskTuningConfig implements AppenderatorConfig
|
||||
{
|
||||
private static final boolean DEFAULT_RESET_OFFSET_AUTOMATICALLY = false;
|
||||
private static final boolean DEFAULT_SKIP_SEQUENCE_NUMBER_AVAILABILITY_CHECK = false;
|
||||
private static final Period DEFAULT_INTERMEDIATE_PERSIST_PERIOD = new Period("PT10M");
|
||||
private static final IndexSpec DEFAULT_INDEX_SPEC = IndexSpec.DEFAULT;
|
||||
private static final Boolean DEFAULT_REPORT_PARSE_EXCEPTIONS = Boolean.FALSE;
|
||||
private static final long DEFAULT_HANDOFF_CONDITION_TIMEOUT = Duration.ofMinutes(15).toMillis();
|
||||
|
||||
private final AppendableIndexSpec appendableIndexSpec;
|
||||
private final int maxRowsInMemory;
|
||||
|
@ -87,30 +91,26 @@ public abstract class SeekableStreamIndexTaskTuningConfig implements Appenderato
|
|||
@Nullable Integer numPersistThreads
|
||||
)
|
||||
{
|
||||
// Cannot be a static because default basePersistDirectory is unique per-instance
|
||||
final RealtimeTuningConfig defaults = RealtimeTuningConfig.makeDefaultTuningConfig(basePersistDirectory);
|
||||
|
||||
this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec;
|
||||
this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory;
|
||||
this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY_REALTIME : maxRowsInMemory;
|
||||
this.partitionsSpec = new DynamicPartitionsSpec(maxRowsPerSegment, maxTotalRows);
|
||||
// initializing this to 0, it will be lazily initialized to a value
|
||||
// @see #getMaxBytesInMemoryOrDefault()
|
||||
this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory;
|
||||
this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck == null ?
|
||||
DEFAULT_SKIP_BYTES_IN_MEMORY_OVERHEAD_CHECK : skipBytesInMemoryOverheadCheck;
|
||||
this.intermediatePersistPeriod = intermediatePersistPeriod == null
|
||||
? defaults.getIntermediatePersistPeriod()
|
||||
: intermediatePersistPeriod;
|
||||
this.intermediatePersistPeriod =
|
||||
intermediatePersistPeriod == null ? DEFAULT_INTERMEDIATE_PERSIST_PERIOD : intermediatePersistPeriod;
|
||||
this.basePersistDirectory = basePersistDirectory;
|
||||
this.maxPendingPersists = maxPendingPersists == null ? 0 : maxPendingPersists;
|
||||
this.indexSpec = indexSpec == null ? defaults.getIndexSpec() : indexSpec;
|
||||
this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec;
|
||||
this.indexSpecForIntermediatePersists = indexSpecForIntermediatePersists == null ?
|
||||
this.indexSpec : indexSpecForIntermediatePersists;
|
||||
this.reportParseExceptions = reportParseExceptions == null
|
||||
? defaults.isReportParseExceptions()
|
||||
? DEFAULT_REPORT_PARSE_EXCEPTIONS
|
||||
: reportParseExceptions;
|
||||
this.handoffConditionTimeout = handoffConditionTimeout == null
|
||||
? defaults.getHandoffConditionTimeout()
|
||||
? DEFAULT_HANDOFF_CONDITION_TIMEOUT
|
||||
: handoffConditionTimeout;
|
||||
this.resetOffsetAutomatically = resetOffsetAutomatically == null
|
||||
? DEFAULT_RESET_OFFSET_AUTOMATICALLY
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.druid.java.util.common.IAE;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
import javax.annotation.concurrent.NotThreadSafe;
|
||||
|
||||
import java.io.InputStream;
|
||||
import java.net.URI;
|
||||
import java.nio.ByteBuffer;
|
||||
|
|
|
@ -27,7 +27,7 @@ import org.apache.druid.java.util.emitter.service.ServiceEventBuilder;
|
|||
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
|
||||
import org.apache.druid.query.DruidMetrics;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||
import org.apache.druid.segment.realtime.FireDepartment;
|
||||
import org.apache.druid.segment.realtime.SegmentGenerationMetrics;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -54,7 +54,7 @@ public class TaskRealtimeMetricsMonitorTest
|
|||
private static final Map<String, Object> TAGS = ImmutableMap.of("author", "Author Name", "version", 10);
|
||||
|
||||
@Mock(answer = Answers.RETURNS_MOCKS)
|
||||
private FireDepartment fireDepartment;
|
||||
private SegmentGenerationMetrics segmentGenerationMetrics;
|
||||
@Mock(answer = Answers.RETURNS_MOCKS)
|
||||
private RowIngestionMeters rowIngestionMeters;
|
||||
@Mock
|
||||
|
@ -74,7 +74,7 @@ public class TaskRealtimeMetricsMonitorTest
|
|||
return null;
|
||||
})
|
||||
.when(emitter).emit(ArgumentMatchers.any(Event.class));
|
||||
target = new TaskRealtimeMetricsMonitor(fireDepartment, rowIngestionMeters, DIMENSIONS, TAGS);
|
||||
target = new TaskRealtimeMetricsMonitor(segmentGenerationMetrics, rowIngestionMeters, DIMENSIONS, TAGS);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -89,7 +89,7 @@ public class TaskRealtimeMetricsMonitorTest
|
|||
@Test
|
||||
public void testdoMonitorWithoutTagsShouldNotEmitTags()
|
||||
{
|
||||
target = new TaskRealtimeMetricsMonitor(fireDepartment, rowIngestionMeters, DIMENSIONS, null);
|
||||
target = new TaskRealtimeMetricsMonitor(segmentGenerationMetrics, rowIngestionMeters, DIMENSIONS, null);
|
||||
for (ServiceMetricEvent sme : emittedEvents.values()) {
|
||||
Assert.assertFalse(sme.getUserDims().containsKey(DruidMetrics.TAGS));
|
||||
}
|
||||
|
|
|
@ -33,7 +33,6 @@ import org.junit.BeforeClass;
|
|||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
|
@ -51,7 +50,7 @@ public class RetrieveSegmentsActionsTest
|
|||
private static Set<DataSegment> expectedUsedSegments;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() throws IOException
|
||||
public static void setup()
|
||||
{
|
||||
task = NoopTask.create();
|
||||
|
||||
|
|
|
@ -60,7 +60,6 @@ import org.junit.Test;
|
|||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.Duration;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
|
@ -404,7 +403,7 @@ public class SegmentAllocateActionTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testSegmentIsAllocatedForLatestUsedSegmentVersion() throws IOException
|
||||
public void testSegmentIsAllocatedForLatestUsedSegmentVersion()
|
||||
{
|
||||
final Task task = NoopTask.create();
|
||||
taskActionTestKit.getTaskLockbox().add(task);
|
||||
|
@ -636,7 +635,7 @@ public class SegmentAllocateActionTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testAddToExistingLinearShardSpecsSameGranularity() throws Exception
|
||||
public void testAddToExistingLinearShardSpecsSameGranularity()
|
||||
{
|
||||
final Task task = NoopTask.create();
|
||||
|
||||
|
@ -702,7 +701,7 @@ public class SegmentAllocateActionTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testAddToExistingNumberedShardSpecsSameGranularity() throws Exception
|
||||
public void testAddToExistingNumberedShardSpecsSameGranularity()
|
||||
{
|
||||
final Task task = NoopTask.create();
|
||||
|
||||
|
@ -766,7 +765,7 @@ public class SegmentAllocateActionTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testAddToExistingNumberedShardSpecsCoarserPreferredGranularity() throws Exception
|
||||
public void testAddToExistingNumberedShardSpecsCoarserPreferredGranularity()
|
||||
{
|
||||
final Task task = NoopTask.create();
|
||||
|
||||
|
@ -806,7 +805,7 @@ public class SegmentAllocateActionTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testAddToExistingNumberedShardSpecsFinerPreferredGranularity() throws Exception
|
||||
public void testAddToExistingNumberedShardSpecsFinerPreferredGranularity()
|
||||
{
|
||||
final Task task = NoopTask.create();
|
||||
|
||||
|
@ -846,7 +845,7 @@ public class SegmentAllocateActionTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testCannotAddToExistingNumberedShardSpecsWithCoarserQueryGranularity() throws Exception
|
||||
public void testCannotAddToExistingNumberedShardSpecsWithCoarserQueryGranularity()
|
||||
{
|
||||
final Task task = NoopTask.create();
|
||||
|
||||
|
@ -889,7 +888,7 @@ public class SegmentAllocateActionTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testWithPartialShardSpecAndOvershadowingSegments() throws IOException
|
||||
public void testWithPartialShardSpecAndOvershadowingSegments()
|
||||
{
|
||||
final Task task = NoopTask.create();
|
||||
taskActionTestKit.getTaskLockbox().add(task);
|
||||
|
@ -1065,7 +1064,7 @@ public class SegmentAllocateActionTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testSegmentIdMustNotBeReused() throws IOException
|
||||
public void testSegmentIdMustNotBeReused()
|
||||
{
|
||||
final IndexerMetadataStorageCoordinator coordinator = taskActionTestKit.getMetadataStorageCoordinator();
|
||||
final TaskLockbox lockbox = taskActionTestKit.getTaskLockbox();
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -1,553 +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.task;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
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.indexer.partitions.PartitionsSpec;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.FileUtils;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
import org.apache.druid.java.util.emitter.core.NoopEmitter;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.IndexMerger;
|
||||
import org.apache.druid.segment.IndexMergerV9;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.incremental.AppendableIndexSpec;
|
||||
import org.apache.druid.segment.incremental.ParseExceptionHandler;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||
import org.apache.druid.segment.incremental.SimpleRowIngestionMeters;
|
||||
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.loading.DataSegmentPusher;
|
||||
import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||
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.Appenderators;
|
||||
import org.apache.druid.segment.realtime.appenderator.BatchAppenderator;
|
||||
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.joda.time.Period;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
|
||||
|
||||
public class AppenderatorsTest
|
||||
{
|
||||
@Test
|
||||
public void testOpenSegmentsOfflineAppenderator() throws Exception
|
||||
{
|
||||
try (final AppenderatorTester tester = new AppenderatorTester("OPEN_SEGMENTS")) {
|
||||
Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl);
|
||||
AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator;
|
||||
Assert.assertTrue(appenderator.isOpenSegments());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClosedSegmentsOfflineAppenderator() throws Exception
|
||||
{
|
||||
try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS")) {
|
||||
Assert.assertTrue(tester.appenderator instanceof AppenderatorImpl);
|
||||
AppenderatorImpl appenderator = (AppenderatorImpl) tester.appenderator;
|
||||
Assert.assertFalse(appenderator.isOpenSegments());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testClosedSegmentsSinksOfflineAppenderator() throws Exception
|
||||
{
|
||||
try (final AppenderatorTester tester = new AppenderatorTester("CLOSED_SEGMENTS_SINKS")) {
|
||||
Assert.assertTrue(tester.appenderator instanceof BatchAppenderator);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private static class AppenderatorTester implements AutoCloseable
|
||||
{
|
||||
public static final String DATASOURCE = "foo";
|
||||
|
||||
private final DataSchema schema;
|
||||
private final AppenderatorConfig tuningConfig;
|
||||
private final FireDepartmentMetrics metrics;
|
||||
private final ObjectMapper objectMapper;
|
||||
private final Appenderator appenderator;
|
||||
private final ServiceEmitter emitter;
|
||||
|
||||
private final List<DataSegment> pushedSegments = new CopyOnWriteArrayList<>();
|
||||
|
||||
|
||||
public AppenderatorTester(
|
||||
final String batchMode
|
||||
)
|
||||
{
|
||||
this(100, 100, null, false, new SimpleRowIngestionMeters(),
|
||||
false, batchMode
|
||||
);
|
||||
}
|
||||
|
||||
public AppenderatorTester(
|
||||
final int maxRowsInMemory,
|
||||
final long maxSizeInBytes,
|
||||
@Nullable final File basePersistDirectory,
|
||||
final boolean enablePushFailure,
|
||||
final RowIngestionMeters rowIngestionMeters,
|
||||
final boolean skipBytesInMemoryOverheadCheck,
|
||||
String batchMode
|
||||
)
|
||||
{
|
||||
objectMapper = new DefaultObjectMapper();
|
||||
objectMapper.registerSubtypes(LinearShardSpec.class);
|
||||
|
||||
final Map<String, Object> parserMap = objectMapper.convertValue(
|
||||
new MapInputRowParser(
|
||||
new JSONParseSpec(
|
||||
new TimestampSpec("ts", "auto", null),
|
||||
DimensionsSpec.EMPTY,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
),
|
||||
Map.class
|
||||
);
|
||||
|
||||
schema = new DataSchema(
|
||||
DATASOURCE,
|
||||
null,
|
||||
null,
|
||||
new AggregatorFactory[]{
|
||||
new CountAggregatorFactory("count"),
|
||||
new LongSumAggregatorFactory("met", "met")
|
||||
},
|
||||
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null),
|
||||
null,
|
||||
parserMap,
|
||||
objectMapper
|
||||
);
|
||||
|
||||
tuningConfig = new TestIndexTuningConfig(
|
||||
TuningConfig.DEFAULT_APPENDABLE_INDEX,
|
||||
maxRowsInMemory,
|
||||
maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes,
|
||||
skipBytesInMemoryOverheadCheck,
|
||||
IndexSpec.DEFAULT,
|
||||
0,
|
||||
false,
|
||||
0L,
|
||||
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
|
||||
IndexMerger.UNLIMITED_MAX_COLUMNS_TO_MERGE,
|
||||
basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory,
|
||||
null
|
||||
);
|
||||
metrics = new FireDepartmentMetrics();
|
||||
|
||||
IndexIO indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT);
|
||||
IndexMergerV9 indexMerger = new IndexMergerV9(
|
||||
objectMapper,
|
||||
indexIO,
|
||||
OffHeapMemorySegmentWriteOutMediumFactory.instance()
|
||||
);
|
||||
|
||||
emitter = new ServiceEmitter(
|
||||
"test",
|
||||
"test",
|
||||
new NoopEmitter()
|
||||
);
|
||||
emitter.start();
|
||||
EmittingLogger.registerEmitter(emitter);
|
||||
DataSegmentPusher dataSegmentPusher = new DataSegmentPusher()
|
||||
{
|
||||
private boolean mustFail = true;
|
||||
|
||||
@Deprecated
|
||||
@Override
|
||||
public String getPathForHadoop(String dataSource)
|
||||
{
|
||||
return getPathForHadoop();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPathForHadoop()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException
|
||||
{
|
||||
if (enablePushFailure && mustFail) {
|
||||
mustFail = false;
|
||||
throw new IOException("Push failure test");
|
||||
} else if (enablePushFailure) {
|
||||
mustFail = true;
|
||||
}
|
||||
pushedSegments.add(segment);
|
||||
return segment;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> makeLoadSpec(URI uri)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
|
||||
switch (batchMode) {
|
||||
case "OPEN_SEGMENTS":
|
||||
appenderator = Appenderators.createOpenSegmentsOffline(
|
||||
schema.getDataSource(),
|
||||
schema,
|
||||
tuningConfig,
|
||||
metrics,
|
||||
dataSegmentPusher,
|
||||
objectMapper,
|
||||
indexIO,
|
||||
indexMerger,
|
||||
rowIngestionMeters,
|
||||
new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0),
|
||||
false,
|
||||
CentralizedDatasourceSchemaConfig.create()
|
||||
);
|
||||
break;
|
||||
case "CLOSED_SEGMENTS":
|
||||
appenderator = Appenderators.createClosedSegmentsOffline(
|
||||
schema.getDataSource(),
|
||||
schema,
|
||||
tuningConfig,
|
||||
metrics,
|
||||
dataSegmentPusher,
|
||||
objectMapper,
|
||||
indexIO,
|
||||
indexMerger,
|
||||
rowIngestionMeters,
|
||||
new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0),
|
||||
false,
|
||||
CentralizedDatasourceSchemaConfig.create()
|
||||
);
|
||||
|
||||
break;
|
||||
case "CLOSED_SEGMENTS_SINKS":
|
||||
appenderator = Appenderators.createOffline(
|
||||
schema.getDataSource(),
|
||||
schema,
|
||||
tuningConfig,
|
||||
metrics,
|
||||
dataSegmentPusher,
|
||||
objectMapper,
|
||||
indexIO,
|
||||
indexMerger,
|
||||
rowIngestionMeters,
|
||||
new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0),
|
||||
false,
|
||||
CentralizedDatasourceSchemaConfig.create()
|
||||
);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Unrecognized batchMode: " + batchMode);
|
||||
}
|
||||
}
|
||||
|
||||
private long getDefaultMaxBytesInMemory()
|
||||
{
|
||||
return (Runtime.getRuntime().totalMemory()) / 3;
|
||||
}
|
||||
|
||||
public DataSchema getSchema()
|
||||
{
|
||||
return schema;
|
||||
}
|
||||
|
||||
public AppenderatorConfig getTuningConfig()
|
||||
{
|
||||
return tuningConfig;
|
||||
}
|
||||
|
||||
public FireDepartmentMetrics getMetrics()
|
||||
{
|
||||
return metrics;
|
||||
}
|
||||
|
||||
public ObjectMapper getObjectMapper()
|
||||
{
|
||||
return objectMapper;
|
||||
}
|
||||
|
||||
public Appenderator getAppenderator()
|
||||
{
|
||||
return appenderator;
|
||||
}
|
||||
|
||||
public List<DataSegment> getPushedSegments()
|
||||
{
|
||||
return pushedSegments;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws Exception
|
||||
{
|
||||
appenderator.close();
|
||||
emitter.close();
|
||||
FileUtils.deleteDirectory(tuningConfig.getBasePersistDirectory());
|
||||
}
|
||||
|
||||
private static File createNewBasePersistDirectory()
|
||||
{
|
||||
return FileUtils.createTempDir("druid-batch-persist");
|
||||
}
|
||||
|
||||
|
||||
static class TestIndexTuningConfig implements AppenderatorConfig
|
||||
{
|
||||
private final AppendableIndexSpec appendableIndexSpec;
|
||||
private final int maxRowsInMemory;
|
||||
private final long maxBytesInMemory;
|
||||
private final boolean skipBytesInMemoryOverheadCheck;
|
||||
private final int maxColumnsToMerge;
|
||||
private final PartitionsSpec partitionsSpec;
|
||||
private final IndexSpec indexSpec;
|
||||
private final File basePersistDirectory;
|
||||
private final int maxPendingPersists;
|
||||
private final boolean reportParseExceptions;
|
||||
private final long pushTimeout;
|
||||
private final IndexSpec indexSpecForIntermediatePersists;
|
||||
@Nullable
|
||||
private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory;
|
||||
private final int numPersistThreads;
|
||||
|
||||
public TestIndexTuningConfig(
|
||||
AppendableIndexSpec appendableIndexSpec,
|
||||
Integer maxRowsInMemory,
|
||||
Long maxBytesInMemory,
|
||||
Boolean skipBytesInMemoryOverheadCheck,
|
||||
IndexSpec indexSpec,
|
||||
Integer maxPendingPersists,
|
||||
Boolean reportParseExceptions,
|
||||
Long pushTimeout,
|
||||
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
|
||||
Integer maxColumnsToMerge,
|
||||
File basePersistDirectory,
|
||||
Integer numPersistThreads
|
||||
)
|
||||
{
|
||||
this.appendableIndexSpec = appendableIndexSpec;
|
||||
this.maxRowsInMemory = maxRowsInMemory;
|
||||
this.maxBytesInMemory = maxBytesInMemory;
|
||||
this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck;
|
||||
this.indexSpec = indexSpec;
|
||||
this.maxPendingPersists = maxPendingPersists;
|
||||
this.reportParseExceptions = reportParseExceptions;
|
||||
this.pushTimeout = pushTimeout;
|
||||
this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory;
|
||||
this.maxColumnsToMerge = maxColumnsToMerge;
|
||||
this.basePersistDirectory = basePersistDirectory;
|
||||
|
||||
this.partitionsSpec = null;
|
||||
this.indexSpecForIntermediatePersists = this.indexSpec;
|
||||
|
||||
this.numPersistThreads = numPersistThreads == null ? DEFAULT_NUM_PERSIST_THREADS : numPersistThreads;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TestIndexTuningConfig withBasePersistDirectory(File dir)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public AppendableIndexSpec getAppendableIndexSpec()
|
||||
{
|
||||
return appendableIndexSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxRowsInMemory()
|
||||
{
|
||||
return maxRowsInMemory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMaxBytesInMemory()
|
||||
{
|
||||
return maxBytesInMemory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isSkipBytesInMemoryOverheadCheck()
|
||||
{
|
||||
return skipBytesInMemoryOverheadCheck;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public PartitionsSpec getPartitionsSpec()
|
||||
{
|
||||
return partitionsSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexSpec getIndexSpec()
|
||||
{
|
||||
return indexSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexSpec getIndexSpecForIntermediatePersists()
|
||||
{
|
||||
return indexSpecForIntermediatePersists;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxPendingPersists()
|
||||
{
|
||||
return maxPendingPersists;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isReportParseExceptions()
|
||||
{
|
||||
return reportParseExceptions;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory()
|
||||
{
|
||||
return segmentWriteOutMediumFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxColumnsToMerge()
|
||||
{
|
||||
return maxColumnsToMerge;
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getBasePersistDirectory()
|
||||
{
|
||||
return basePersistDirectory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Period getIntermediatePersistPeriod()
|
||||
{
|
||||
return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumPersistThreads()
|
||||
{
|
||||
return numPersistThreads;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
TestIndexTuningConfig that = (TestIndexTuningConfig) o;
|
||||
return Objects.equals(appendableIndexSpec, that.appendableIndexSpec) &&
|
||||
maxRowsInMemory == that.maxRowsInMemory &&
|
||||
maxBytesInMemory == that.maxBytesInMemory &&
|
||||
skipBytesInMemoryOverheadCheck == that.skipBytesInMemoryOverheadCheck &&
|
||||
maxColumnsToMerge == that.maxColumnsToMerge &&
|
||||
maxPendingPersists == that.maxPendingPersists &&
|
||||
reportParseExceptions == that.reportParseExceptions &&
|
||||
pushTimeout == that.pushTimeout &&
|
||||
numPersistThreads == that.numPersistThreads &&
|
||||
Objects.equals(partitionsSpec, that.partitionsSpec) &&
|
||||
Objects.equals(indexSpec, that.indexSpec) &&
|
||||
Objects.equals(indexSpecForIntermediatePersists, that.indexSpecForIntermediatePersists) &&
|
||||
Objects.equals(basePersistDirectory, that.basePersistDirectory) &&
|
||||
Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(
|
||||
appendableIndexSpec,
|
||||
maxRowsInMemory,
|
||||
maxBytesInMemory,
|
||||
skipBytesInMemoryOverheadCheck,
|
||||
maxColumnsToMerge,
|
||||
partitionsSpec,
|
||||
indexSpec,
|
||||
indexSpecForIntermediatePersists,
|
||||
basePersistDirectory,
|
||||
maxPendingPersists,
|
||||
reportParseExceptions,
|
||||
pushTimeout,
|
||||
segmentWriteOutMediumFactory,
|
||||
numPersistThreads
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "IndexTuningConfig{" +
|
||||
"maxRowsInMemory=" + maxRowsInMemory +
|
||||
", maxBytesInMemory=" + maxBytesInMemory +
|
||||
", skipBytesInMemoryOverheadCheck=" + skipBytesInMemoryOverheadCheck +
|
||||
", maxColumnsToMerge=" + maxColumnsToMerge +
|
||||
", partitionsSpec=" + partitionsSpec +
|
||||
", indexSpec=" + indexSpec +
|
||||
", indexSpecForIntermediatePersists=" + indexSpecForIntermediatePersists +
|
||||
", basePersistDirectory=" + basePersistDirectory +
|
||||
", maxPendingPersists=" + maxPendingPersists +
|
||||
", reportParseExceptions=" + reportParseExceptions +
|
||||
", pushTimeout=" + pushTimeout +
|
||||
", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory +
|
||||
", numPersistThreads=" + numPersistThreads +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -52,7 +52,7 @@ import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
|||
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.FireDepartmentMetrics;
|
||||
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;
|
||||
|
@ -114,7 +114,7 @@ public class BatchAppenderatorsTest
|
|||
|
||||
private final DataSchema schema;
|
||||
private final AppenderatorConfig tuningConfig;
|
||||
private final FireDepartmentMetrics metrics;
|
||||
private final SegmentGenerationMetrics metrics;
|
||||
private final ObjectMapper objectMapper;
|
||||
private final Appenderator appenderator;
|
||||
private final ServiceEmitter emitter;
|
||||
|
@ -185,7 +185,7 @@ public class BatchAppenderatorsTest
|
|||
basePersistDirectory == null ? createNewBasePersistDirectory() : basePersistDirectory,
|
||||
null
|
||||
);
|
||||
metrics = new FireDepartmentMetrics();
|
||||
metrics = new SegmentGenerationMetrics();
|
||||
|
||||
IndexIO indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT);
|
||||
IndexMergerV9 indexMerger = new IndexMergerV9(
|
||||
|
@ -329,7 +329,7 @@ public class BatchAppenderatorsTest
|
|||
return tuningConfig;
|
||||
}
|
||||
|
||||
public FireDepartmentMetrics getMetrics()
|
||||
public SegmentGenerationMetrics getMetrics()
|
||||
{
|
||||
return metrics;
|
||||
}
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.exc.ValueInstantiationException;
|
||||
import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
|
@ -122,7 +121,6 @@ import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
|||
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
|
||||
import org.apache.druid.segment.indexing.BatchIOConfig;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.RealtimeTuningConfig;
|
||||
import org.apache.druid.segment.indexing.TuningConfig;
|
||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
import org.apache.druid.segment.join.NoopJoinableFactory;
|
||||
|
@ -884,48 +882,6 @@ public class CompactionTaskTest
|
|||
Assert.assertEquals(compactionTuningConfig, CompactionTask.getTuningConfig(parallelIndexTuningConfig));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSerdeWithUnknownTuningConfigThrowingError() throws IOException
|
||||
{
|
||||
final OldCompactionTaskWithAnyTuningConfigType taskWithUnknownTuningConfig =
|
||||
new OldCompactionTaskWithAnyTuningConfigType(
|
||||
null,
|
||||
null,
|
||||
DATA_SOURCE,
|
||||
null,
|
||||
SEGMENTS,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
RealtimeTuningConfig.makeDefaultTuningConfig(null),
|
||||
null,
|
||||
OBJECT_MAPPER,
|
||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
null,
|
||||
toolbox.getRowIngestionMetersFactory(),
|
||||
COORDINATOR_CLIENT,
|
||||
segmentCacheManagerFactory,
|
||||
RETRY_POLICY_FACTORY,
|
||||
toolbox.getAppenderatorsManager()
|
||||
);
|
||||
|
||||
final ObjectMapper mapper = new DefaultObjectMapper((DefaultObjectMapper) OBJECT_MAPPER);
|
||||
mapper.registerSubtypes(
|
||||
new NamedType(OldCompactionTaskWithAnyTuningConfigType.class, "compact"),
|
||||
new NamedType(RealtimeTuningConfig.class, "realtime")
|
||||
);
|
||||
final byte[] bytes = mapper.writeValueAsBytes(taskWithUnknownTuningConfig);
|
||||
|
||||
expectedException.expect(ValueInstantiationException.class);
|
||||
expectedException.expectCause(CoreMatchers.instanceOf(IllegalStateException.class));
|
||||
expectedException.expectMessage(
|
||||
"Unknown tuningConfig type: [org.apache.druid.segment.indexing.RealtimeTuningConfig]"
|
||||
);
|
||||
mapper.readValue(bytes, CompactionTask.class);
|
||||
}
|
||||
|
||||
private static void assertEquals(CompactionTask expected, CompactionTask actual)
|
||||
{
|
||||
Assert.assertEquals(expected.getType(), actual.getType());
|
||||
|
|
|
@ -79,6 +79,7 @@ import org.apache.druid.segment.VirtualColumns;
|
|||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.data.CompressionStrategy;
|
||||
import org.apache.druid.segment.handoff.NoopSegmentHandoffNotifierFactory;
|
||||
import org.apache.druid.segment.handoff.SegmentHandoffNotifier;
|
||||
import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||
|
@ -93,7 +94,6 @@ 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.plumber.NoopSegmentHandoffNotifierFactory;
|
||||
import org.apache.druid.segment.transform.ExpressionTransform;
|
||||
import org.apache.druid.segment.transform.TransformSpec;
|
||||
import org.apache.druid.server.metrics.NoopServiceEmitter;
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -47,12 +47,8 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
|
|||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
||||
import org.apache.druid.segment.indexing.RealtimeTuningConfig;
|
||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
import org.apache.druid.segment.realtime.FireDepartment;
|
||||
import org.apache.druid.server.security.AuthTestUtils;
|
||||
import org.apache.druid.timeline.partition.NoneShardSpec;
|
||||
import org.hamcrest.CoreMatchers;
|
||||
import org.joda.time.Period;
|
||||
import org.junit.Assert;
|
||||
|
@ -387,92 +383,6 @@ public class TaskSerdeTest
|
|||
Assert.assertTrue(task2.getIngestionSchema().getIOConfig().getInputSource() instanceof LocalInputSource);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRealtimeIndexTaskSerde() throws Exception
|
||||
{
|
||||
|
||||
final RealtimeIndexTask task = new RealtimeIndexTask(
|
||||
null,
|
||||
new TaskResource("rofl", 2),
|
||||
new FireDepartment(
|
||||
new DataSchema(
|
||||
"foo",
|
||||
null,
|
||||
new AggregatorFactory[0],
|
||||
new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null),
|
||||
null,
|
||||
jsonMapper
|
||||
),
|
||||
new RealtimeIOConfig(
|
||||
new MockFirehoseFactory(),
|
||||
(schema, config, metrics) -> null
|
||||
),
|
||||
|
||||
new RealtimeTuningConfig(
|
||||
null,
|
||||
1,
|
||||
10L,
|
||||
null,
|
||||
new Period("PT10M"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
1,
|
||||
NoneShardSpec.instance(),
|
||||
indexSpec,
|
||||
null,
|
||||
0,
|
||||
0,
|
||||
true,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
),
|
||||
null
|
||||
);
|
||||
|
||||
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
|
||||
final RealtimeIndexTask task2 = (RealtimeIndexTask) jsonMapper.readValue(json, Task.class);
|
||||
|
||||
Assert.assertEquals("foo", task.getDataSource());
|
||||
Assert.assertEquals(2, task.getTaskResource().getRequiredCapacity());
|
||||
Assert.assertEquals("rofl", task.getTaskResource().getAvailabilityGroup());
|
||||
Assert.assertEquals(
|
||||
new Period("PT10M"),
|
||||
task.getRealtimeIngestionSchema()
|
||||
.getTuningConfig().getWindowPeriod()
|
||||
);
|
||||
Assert.assertEquals(
|
||||
Granularities.HOUR,
|
||||
task.getRealtimeIngestionSchema().getDataSchema().getGranularitySpec().getSegmentGranularity()
|
||||
);
|
||||
Assert.assertTrue(task.getRealtimeIngestionSchema().getTuningConfig().isReportParseExceptions());
|
||||
|
||||
Assert.assertEquals(task.getId(), task2.getId());
|
||||
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
|
||||
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
|
||||
Assert.assertEquals(task.getTaskResource().getRequiredCapacity(), task2.getTaskResource().getRequiredCapacity());
|
||||
Assert.assertEquals(task.getTaskResource().getAvailabilityGroup(), task2.getTaskResource().getAvailabilityGroup());
|
||||
Assert.assertEquals(
|
||||
task.getRealtimeIngestionSchema().getTuningConfig().getWindowPeriod(),
|
||||
task2.getRealtimeIngestionSchema().getTuningConfig().getWindowPeriod()
|
||||
);
|
||||
Assert.assertEquals(
|
||||
task.getRealtimeIngestionSchema().getTuningConfig().getMaxBytesInMemory(),
|
||||
task2.getRealtimeIngestionSchema().getTuningConfig().getMaxBytesInMemory()
|
||||
);
|
||||
Assert.assertEquals(
|
||||
task.getRealtimeIngestionSchema().getDataSchema().getGranularitySpec().getSegmentGranularity(),
|
||||
task2.getRealtimeIngestionSchema().getDataSchema().getGranularitySpec().getSegmentGranularity()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArchiveTaskSerde() throws Exception
|
||||
{
|
||||
|
|
|
@ -38,7 +38,7 @@ import org.apache.druid.segment.join.JoinableFactory;
|
|||
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.FireDepartmentMetrics;
|
||||
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.Appenderators;
|
||||
|
@ -56,7 +56,7 @@ public class TestAppenderatorsManager implements AppenderatorsManager
|
|||
String taskId,
|
||||
DataSchema schema,
|
||||
AppenderatorConfig config,
|
||||
FireDepartmentMetrics metrics,
|
||||
SegmentGenerationMetrics metrics,
|
||||
DataSegmentPusher dataSegmentPusher,
|
||||
ObjectMapper objectMapper,
|
||||
IndexIO indexIO,
|
||||
|
@ -105,7 +105,7 @@ public class TestAppenderatorsManager implements AppenderatorsManager
|
|||
String taskId,
|
||||
DataSchema schema,
|
||||
AppenderatorConfig config,
|
||||
FireDepartmentMetrics metrics,
|
||||
SegmentGenerationMetrics metrics,
|
||||
DataSegmentPusher dataSegmentPusher,
|
||||
ObjectMapper objectMapper,
|
||||
IndexIO indexIO,
|
||||
|
@ -137,7 +137,7 @@ public class TestAppenderatorsManager implements AppenderatorsManager
|
|||
String taskId,
|
||||
DataSchema schema,
|
||||
AppenderatorConfig config,
|
||||
FireDepartmentMetrics metrics,
|
||||
SegmentGenerationMetrics metrics,
|
||||
DataSegmentPusher dataSegmentPusher,
|
||||
ObjectMapper objectMapper,
|
||||
IndexIO indexIO,
|
||||
|
@ -169,7 +169,7 @@ public class TestAppenderatorsManager implements AppenderatorsManager
|
|||
String taskId,
|
||||
DataSchema schema,
|
||||
AppenderatorConfig config,
|
||||
FireDepartmentMetrics metrics,
|
||||
SegmentGenerationMetrics metrics,
|
||||
DataSegmentPusher dataSegmentPusher,
|
||||
ObjectMapper objectMapper,
|
||||
IndexIO indexIO,
|
||||
|
|
|
@ -47,9 +47,7 @@ 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.MapInputRowParser;
|
||||
import org.apache.druid.data.input.impl.NoopInputFormat;
|
||||
import org.apache.druid.data.input.impl.TimeAndDimsParseSpec;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.discovery.DataNodeService;
|
||||
import org.apache.druid.discovery.DruidNodeAnnouncer;
|
||||
|
@ -83,7 +81,6 @@ import org.apache.druid.indexing.common.task.KillUnusedSegmentsTask;
|
|||
import org.apache.druid.indexing.common.task.NoopTask;
|
||||
import org.apache.druid.indexing.common.task.NoopTaskContextEnricher;
|
||||
import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter;
|
||||
import org.apache.druid.indexing.common.task.RealtimeIndexTask;
|
||||
import org.apache.druid.indexing.common.task.Task;
|
||||
import org.apache.druid.indexing.common.task.TaskResource;
|
||||
import org.apache.druid.indexing.common.task.TestAppenderatorsManager;
|
||||
|
@ -103,15 +100,12 @@ import org.apache.druid.java.util.common.Intervals;
|
|||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.RE;
|
||||
import org.apache.druid.java.util.common.Stopwatch;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.guava.Comparators;
|
||||
import org.apache.druid.java.util.common.jackson.JacksonUtils;
|
||||
import org.apache.druid.java.util.common.parsers.CloseableIterator;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||
import org.apache.druid.java.util.metrics.Monitor;
|
||||
import org.apache.druid.java.util.metrics.MonitorScheduler;
|
||||
import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory;
|
||||
import org.apache.druid.metadata.TestDerbyConnector;
|
||||
|
@ -121,18 +115,14 @@ import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
|||
import org.apache.druid.query.SegmentDescriptor;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.IndexMergerV9Factory;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.SegmentSchemaMapping;
|
||||
import org.apache.druid.segment.TestHelper;
|
||||
import org.apache.druid.segment.TestIndex;
|
||||
import org.apache.druid.segment.handoff.SegmentHandoffNotifier;
|
||||
import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
||||
import org.apache.druid.segment.indexing.RealtimeTuningConfig;
|
||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapperTest;
|
||||
import org.apache.druid.segment.join.NoopJoinableFactory;
|
||||
|
@ -141,8 +131,6 @@ import org.apache.druid.segment.loading.LocalDataSegmentKiller;
|
|||
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.FireDepartment;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentTest;
|
||||
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
|
||||
import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager;
|
||||
import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider;
|
||||
|
@ -651,7 +639,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
|
|||
MAPPER,
|
||||
INDEX_IO,
|
||||
MapCache.create(0),
|
||||
FireDepartmentTest.NO_CACHE_CONFIG,
|
||||
new CacheConfig(),
|
||||
new CachePopulatorStats(),
|
||||
INDEX_MERGER_V9_FACTORY,
|
||||
EasyMock.createNiceMock(DruidNodeAnnouncer.class),
|
||||
|
@ -1251,114 +1239,6 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
|
|||
Assert.assertEquals("segments nuked", 0, mdc.getNuked().size());
|
||||
}
|
||||
|
||||
@Test(timeout = 60_000L)
|
||||
public void testRealtimeIndexTask() throws Exception
|
||||
{
|
||||
publishCountDown = new CountDownLatch(1);
|
||||
monitorScheduler.addMonitor(EasyMock.anyObject(Monitor.class));
|
||||
EasyMock.expectLastCall().times(1);
|
||||
monitorScheduler.removeMonitor(EasyMock.anyObject(Monitor.class));
|
||||
EasyMock.expectLastCall().times(1);
|
||||
EasyMock.replay(monitorScheduler, queryRunnerFactoryConglomerate);
|
||||
|
||||
RealtimeIndexTask realtimeIndexTask = newRealtimeIndexTask();
|
||||
final String taskId = realtimeIndexTask.getId();
|
||||
|
||||
taskQueue.start();
|
||||
taskQueue.add(realtimeIndexTask);
|
||||
//wait for task to process events and publish segment
|
||||
publishCountDown.await();
|
||||
|
||||
// Realtime Task has published the segment, simulate loading of segment to a historical node so that task finishes with SUCCESS status
|
||||
Assert.assertEquals(1, handOffCallbacks.size());
|
||||
Pair<Executor, Runnable> executorRunnablePair = Iterables.getOnlyElement(handOffCallbacks.values());
|
||||
executorRunnablePair.lhs.execute(executorRunnablePair.rhs);
|
||||
handOffCallbacks.clear();
|
||||
|
||||
// Wait for realtime index task to handle callback in plumber and succeed
|
||||
while (tsqa.getStatus(taskId).get().isRunnable()) {
|
||||
Thread.sleep(10);
|
||||
}
|
||||
|
||||
TaskStatus status = tsqa.getStatus(taskId).get();
|
||||
Assert.assertTrue("Task should be in Success state", status.isSuccess());
|
||||
Assert.assertEquals(taskLocation, status.getLocation());
|
||||
|
||||
Assert.assertEquals(1, announcedSinks);
|
||||
Assert.assertEquals(1, pushedSegments);
|
||||
Assert.assertEquals(1, mdc.getPublished().size());
|
||||
DataSegment segment = mdc.getPublished().iterator().next();
|
||||
Assert.assertEquals("test_ds", segment.getDataSource());
|
||||
Assert.assertEquals(ImmutableList.of("dim1", "dim2"), segment.getDimensions());
|
||||
Assert.assertEquals(
|
||||
Intervals.of(now.toString("YYYY-MM-dd") + "/" + now.plusDays(1).toString("YYYY-MM-dd")),
|
||||
segment.getInterval()
|
||||
);
|
||||
Assert.assertEquals(ImmutableList.of("count"), segment.getMetrics());
|
||||
EasyMock.verify(monitorScheduler, queryRunnerFactoryConglomerate);
|
||||
}
|
||||
|
||||
@Test(timeout = 60_000L)
|
||||
public void testRealtimeIndexTaskFailure() throws Exception
|
||||
{
|
||||
dataSegmentPusher = new DataSegmentPusher()
|
||||
{
|
||||
@Deprecated
|
||||
@Override
|
||||
public String getPathForHadoop(String s)
|
||||
{
|
||||
return getPathForHadoop();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPathForHadoop()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSegment push(File file, DataSegment dataSegment, boolean useUniquePath)
|
||||
{
|
||||
throw new RuntimeException("FAILURE");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> makeLoadSpec(URI uri)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
|
||||
tb = setUpTaskToolboxFactory(dataSegmentPusher, handoffNotifierFactory, mdc);
|
||||
|
||||
taskRunner = setUpThreadPoolTaskRunner(tb);
|
||||
|
||||
taskQueue = setUpTaskQueue(taskStorage, taskRunner);
|
||||
|
||||
monitorScheduler.addMonitor(EasyMock.anyObject(Monitor.class));
|
||||
EasyMock.expectLastCall().times(1);
|
||||
monitorScheduler.removeMonitor(EasyMock.anyObject(Monitor.class));
|
||||
EasyMock.expectLastCall().times(1);
|
||||
EasyMock.replay(monitorScheduler, queryRunnerFactoryConglomerate);
|
||||
|
||||
RealtimeIndexTask realtimeIndexTask = newRealtimeIndexTask();
|
||||
final String taskId = realtimeIndexTask.getId();
|
||||
|
||||
taskQueue.start();
|
||||
taskQueue.add(realtimeIndexTask);
|
||||
|
||||
// Wait for realtime index task to fail
|
||||
while (tsqa.getStatus(taskId).get().isRunnable()) {
|
||||
Thread.sleep(10);
|
||||
}
|
||||
|
||||
TaskStatus status = tsqa.getStatus(taskId).get();
|
||||
Assert.assertTrue("Task should be in Failure state", status.isFailure());
|
||||
Assert.assertEquals(taskLocation, status.getLocation());
|
||||
|
||||
EasyMock.verify(monitorScheduler, queryRunnerFactoryConglomerate);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testResumeTasks() throws Exception
|
||||
{
|
||||
|
@ -1629,60 +1509,4 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
|
|||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
private RealtimeIndexTask newRealtimeIndexTask()
|
||||
{
|
||||
String taskId = StringUtils.format("rt_task_%s", System.currentTimeMillis());
|
||||
DataSchema dataSchema = new DataSchema(
|
||||
"test_ds",
|
||||
TestHelper.makeJsonMapper().convertValue(
|
||||
new MapInputRowParser(
|
||||
new TimeAndDimsParseSpec(
|
||||
new TimestampSpec(null, null, null),
|
||||
DimensionsSpec.EMPTY
|
||||
)
|
||||
),
|
||||
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
|
||||
),
|
||||
new AggregatorFactory[]{new LongSumAggregatorFactory("count", "rows")},
|
||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
|
||||
null,
|
||||
mapper
|
||||
);
|
||||
RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig(
|
||||
new MockFirehoseFactory(),
|
||||
null
|
||||
// PlumberSchool - Realtime Index Task always uses RealtimePlumber which is hardcoded in RealtimeIndexTask class
|
||||
);
|
||||
RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig(
|
||||
null,
|
||||
1000,
|
||||
null,
|
||||
null,
|
||||
new Period("P1Y"),
|
||||
null, //default window period of 10 minutes
|
||||
null, // base persist dir ignored by Realtime Index task
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
0,
|
||||
0,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
FireDepartment fireDepartment = new FireDepartment(dataSchema, realtimeIOConfig, realtimeTuningConfig);
|
||||
return new RealtimeIndexTask(
|
||||
taskId,
|
||||
new TaskResource(taskId, 1),
|
||||
fireDepartment,
|
||||
null
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,18 +20,16 @@
|
|||
package org.apache.druid.indexing.worker;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.druid.data.input.impl.NoopInputSource;
|
||||
import org.apache.druid.indexer.TaskLocation;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.TestUtils;
|
||||
import org.apache.druid.indexing.common.task.RealtimeIndexTask;
|
||||
import org.apache.druid.indexing.common.task.IndexTask;
|
||||
import org.apache.druid.indexing.common.task.Task;
|
||||
import org.apache.druid.indexing.common.task.TaskResource;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
||||
import org.apache.druid.segment.realtime.FireDepartment;
|
||||
import org.easymock.EasyMock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -48,13 +46,13 @@ public class TaskAnnouncementTest
|
|||
@Test
|
||||
public void testBackwardsCompatibleSerde() throws Exception
|
||||
{
|
||||
final RealtimeIOConfig realtimeIOConfig = EasyMock.createNiceMock(RealtimeIOConfig.class);
|
||||
final Task task = new RealtimeIndexTask(
|
||||
final IndexTask.IndexIOConfig ioConfig = new IndexTask.IndexIOConfig(null, new NoopInputSource(), null, null, null);
|
||||
final Task task = new IndexTask(
|
||||
"theid",
|
||||
new TaskResource("rofl", 2),
|
||||
new FireDepartment(
|
||||
new IndexTask.IndexIngestionSpec(
|
||||
new DataSchema("foo", null, new AggregatorFactory[0], null, null, new DefaultObjectMapper()),
|
||||
realtimeIOConfig,
|
||||
ioConfig,
|
||||
null
|
||||
),
|
||||
null
|
||||
|
|
|
@ -252,7 +252,7 @@ case $CMD in
|
|||
cd $COMPOSE_DIR
|
||||
$DOCKER_COMPOSE $DOCKER_ARGS up -d
|
||||
# Enable the following for debugging
|
||||
#show_status
|
||||
# show_status
|
||||
;;
|
||||
"status" )
|
||||
check_env_file
|
||||
|
|
|
@ -46,28 +46,27 @@ services:
|
|||
kafka:
|
||||
image: bitnami/kafka:${KAFKA_VERSION}
|
||||
container_name: kafka
|
||||
# platform: linux/x86_64
|
||||
labels:
|
||||
druid-int-test: "true"
|
||||
ports:
|
||||
- 9092:9092
|
||||
- 9093:9093
|
||||
- 9094:9094
|
||||
networks:
|
||||
druid-it-net:
|
||||
ipv4_address: 172.172.172.2
|
||||
volumes:
|
||||
- ${SHARED_DIR}/kafka:/bitnami/kafka
|
||||
environment:
|
||||
# This is the default: making it explicit
|
||||
KAFKA_CFG_ZOOKEEPER_CONNECT: zookeeper:2181
|
||||
# Plaintext is disabled by default
|
||||
ALLOW_PLAINTEXT_LISTENER: "yes"
|
||||
# Adapted from base-setup.sh and Bitnami docs
|
||||
KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP: "INTERNAL:PLAINTEXT,EXTERNAL:PLAINTEXT"
|
||||
KAFKA_CFG_LISTENERS: "INTERNAL://:9092,EXTERNAL://:9093"
|
||||
KAFKA_CFG_ADVERTISED_LISTENERS: "INTERNAL://kafka:9092,EXTERNAL://localhost:9093"
|
||||
KAFKA_CFG_INTER_BROKER_LISTENER_NAME: INTERNAL
|
||||
depends_on:
|
||||
- zookeeper
|
||||
- KAFKA_CFG_NODE_ID=1001
|
||||
- KAFKA_CFG_PROCESS_ROLES=controller,broker
|
||||
- KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093
|
||||
- KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT
|
||||
- KAFKA_CFG_CONTROLLER_QUORUM_VOTERS=1001@kafka:9093
|
||||
- KAFKA_CFG_LISTENERS=PLAINTEXT://:9092,CONTROLLER://:9093,EXTERNAL://:9094
|
||||
- KAFKA_CFG_ADVERTISED_LISTENERS=PLAINTEXT://kafka:9092,EXTERNAL://localhost:9094
|
||||
- KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP=CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT
|
||||
- KAFKA_CFG_CONTROLLER_LISTENER_NAMES=CONTROLLER
|
||||
|
||||
# Uses the official MySQL image
|
||||
# See https://hub.docker.com/_/mysql
|
||||
|
|
|
@ -0,0 +1,105 @@
|
|||
# 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.
|
||||
|
||||
networks:
|
||||
druid-it-net:
|
||||
name: druid-it-net
|
||||
ipam:
|
||||
config:
|
||||
- subnet: 172.172.172.0/24
|
||||
|
||||
services:
|
||||
zookeeper:
|
||||
extends:
|
||||
file: ../Common/dependencies.yaml
|
||||
service: zookeeper
|
||||
|
||||
metadata:
|
||||
extends:
|
||||
file: ../Common/dependencies.yaml
|
||||
service: metadata
|
||||
|
||||
coordinator:
|
||||
extends:
|
||||
file: ../Common/druid.yaml
|
||||
service: coordinator
|
||||
container_name: coordinator
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
# The frequency with which the coordinator polls the database
|
||||
# for changes. The DB population code has to wait at least this
|
||||
# long for the coordinator to notice changes.
|
||||
- druid_manager_segments_pollDuration=PT5S
|
||||
- druid_coordinator_period=PT10S
|
||||
depends_on:
|
||||
- zookeeper
|
||||
- metadata
|
||||
|
||||
overlord:
|
||||
extends:
|
||||
file: ../Common/druid.yaml
|
||||
service: overlord
|
||||
container_name: overlord
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
depends_on:
|
||||
- zookeeper
|
||||
- metadata
|
||||
|
||||
broker:
|
||||
extends:
|
||||
file: ../Common/druid.yaml
|
||||
service: broker
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
depends_on:
|
||||
- zookeeper
|
||||
|
||||
router:
|
||||
extends:
|
||||
file: ../Common/druid.yaml
|
||||
service: router
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
depends_on:
|
||||
- zookeeper
|
||||
|
||||
historical:
|
||||
extends:
|
||||
file: ../Common/druid.yaml
|
||||
service: historical
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
depends_on:
|
||||
- zookeeper
|
||||
|
||||
middlemanager:
|
||||
extends:
|
||||
file: ../Common/druid.yaml
|
||||
service: middlemanager
|
||||
environment:
|
||||
- DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP}
|
||||
volumes:
|
||||
# Test data
|
||||
- ../../resources:/resources
|
||||
depends_on:
|
||||
- zookeeper
|
||||
|
||||
kafka:
|
||||
extends:
|
||||
file: ../Common/dependencies.yaml
|
||||
service: kafka
|
||||
depends_on:
|
||||
- zookeeper
|
|
@ -74,6 +74,11 @@
|
|||
<artifactId>druid-indexing-service</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid.extensions</groupId>
|
||||
<artifactId>druid-kafka-indexing-service</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject</groupId>
|
||||
<artifactId>guice</artifactId>
|
||||
|
@ -399,6 +404,15 @@
|
|||
<it.category>AzureDeepStorage</it.category>
|
||||
</properties>
|
||||
</profile>
|
||||
<profile>
|
||||
<id>IT-Query</id>
|
||||
<activation>
|
||||
<activeByDefault>false</activeByDefault>
|
||||
</activation>
|
||||
<properties>
|
||||
<it.category>Query</it.category>
|
||||
</properties>
|
||||
</profile>
|
||||
<profile>
|
||||
<id>IT-MultiStageQuery</id>
|
||||
<activation>
|
||||
|
|
|
@ -17,10 +17,8 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.metadata;
|
||||
package org.apache.druid.testsEx.categories;
|
||||
|
||||
import org.apache.druid.segment.realtime.SegmentPublisher;
|
||||
|
||||
public interface MetadataSegmentPublisher extends SegmentPublisher
|
||||
public class Query
|
||||
{
|
||||
}
|
|
@ -62,13 +62,13 @@ class IntegrationTestingConfigEx implements IntegrationTestingConfig
|
|||
@Override
|
||||
public String getKafkaHost()
|
||||
{
|
||||
return config.requireKafka().instance().clientHost();
|
||||
return config.requireKafka().instance().clientHost() + ":" + config.requireKafka().instance().clientPort();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getKafkaInternalHost()
|
||||
{
|
||||
return config.requireKafka().instance().host();
|
||||
return config.requireKafka().instance().host() + ":" + config.requireKafka().instance().port();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,213 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.testsEx.query;
|
||||
|
||||
import org.apache.druid.indexing.kafka.KafkaConsumerConfigs;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import org.apache.druid.testing.IntegrationTestingConfig;
|
||||
import org.apache.druid.testing.utils.ITRetryUtil;
|
||||
import org.apache.druid.testing.utils.KafkaAdminClient;
|
||||
import org.apache.druid.testing.utils.KafkaEventWriter;
|
||||
import org.apache.druid.testing.utils.KafkaUtil;
|
||||
import org.apache.druid.testing.utils.StreamEventWriter;
|
||||
import org.apache.druid.testsEx.categories.Query;
|
||||
import org.apache.druid.testsEx.config.DruidTestRunner;
|
||||
import org.apache.druid.testsEx.indexer.AbstractIndexerTest;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.runner.RunWith;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.InputStreamReader;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.UUID;
|
||||
import java.util.function.Function;
|
||||
|
||||
@RunWith(DruidTestRunner.class)
|
||||
@Category(Query.class)
|
||||
public class ITUnionQueryTest extends AbstractIndexerTest
|
||||
{
|
||||
private static final Logger LOG = new Logger(ITUnionQueryTest.class);
|
||||
private static final String UNION_SUPERVISOR_TEMPLATE = "/query/union_kafka_supervisor_template.json";
|
||||
private static final String UNION_DATA_FILE = "/query/union_data.json";
|
||||
private static final String UNION_QUERIES_RESOURCE = "/query/union_queries.json";
|
||||
private static final String UNION_DATASOURCE = "wikipedia_index_test";
|
||||
private String fullDatasourceName;
|
||||
|
||||
@Test
|
||||
public void testUnionQuery() throws Exception
|
||||
{
|
||||
fullDatasourceName = UNION_DATASOURCE + config.getExtraDatasourceNameSuffix();
|
||||
final String baseName = fullDatasourceName + UUID.randomUUID();
|
||||
KafkaAdminClient streamAdminClient = new KafkaAdminClient(config);
|
||||
List<String> supervisors = new ArrayList<>();
|
||||
|
||||
final int numDatasources = 3;
|
||||
for (int i = 0; i < numDatasources; i++) {
|
||||
String datasource = baseName + "-" + i;
|
||||
streamAdminClient.createStream(datasource, 1, Collections.emptyMap());
|
||||
ITRetryUtil.retryUntil(
|
||||
() -> streamAdminClient.isStreamActive(datasource),
|
||||
true,
|
||||
10000,
|
||||
30,
|
||||
"Wait for stream active"
|
||||
);
|
||||
String supervisorSpec = generateStreamIngestionPropsTransform(
|
||||
datasource,
|
||||
datasource,
|
||||
config
|
||||
).apply(getResourceAsString(UNION_SUPERVISOR_TEMPLATE));
|
||||
LOG.info("supervisorSpec: [%s]\n", supervisorSpec);
|
||||
// Start supervisor
|
||||
String specResponse = indexer.submitSupervisor(supervisorSpec);
|
||||
LOG.info("Submitted supervisor [%s]", specResponse);
|
||||
supervisors.add(specResponse);
|
||||
|
||||
int ctr = 0;
|
||||
try (
|
||||
StreamEventWriter streamEventWriter = new KafkaEventWriter(config, false);
|
||||
BufferedReader reader = new BufferedReader(
|
||||
new InputStreamReader(getResourceAsStream(UNION_DATA_FILE), StandardCharsets.UTF_8)
|
||||
)
|
||||
) {
|
||||
String line;
|
||||
while ((line = reader.readLine()) != null) {
|
||||
streamEventWriter.write(datasource, StringUtils.toUtf8(line));
|
||||
ctr++;
|
||||
}
|
||||
}
|
||||
final int numWritten = ctr;
|
||||
|
||||
LOG.info("Waiting for stream indexing tasks to consume events");
|
||||
|
||||
ITRetryUtil.retryUntilTrue(
|
||||
() ->
|
||||
numWritten == this.queryHelper.countRows(
|
||||
datasource,
|
||||
Intervals.ETERNITY,
|
||||
name -> new LongSumAggregatorFactory(name, "count")
|
||||
),
|
||||
StringUtils.format(
|
||||
"dataSource[%s] consumed [%,d] events, expected [%,d]",
|
||||
datasource,
|
||||
this.queryHelper.countRows(
|
||||
datasource,
|
||||
Intervals.ETERNITY,
|
||||
name -> new LongSumAggregatorFactory(name, "count")
|
||||
),
|
||||
numWritten
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
String queryResponseTemplate = StringUtils.replace(
|
||||
getResourceAsString(UNION_QUERIES_RESOURCE),
|
||||
"%%DATASOURCE%%",
|
||||
baseName
|
||||
);
|
||||
|
||||
queryHelper.testQueriesFromString(queryResponseTemplate);
|
||||
|
||||
|
||||
for (int i = 0; i < numDatasources; i++) {
|
||||
indexer.terminateSupervisor(supervisors.get(i));
|
||||
streamAdminClient.deleteStream(baseName + "-" + i);
|
||||
}
|
||||
|
||||
for (int i = 0; i < numDatasources; i++) {
|
||||
final int datasourceNumber = i;
|
||||
ITRetryUtil.retryUntil(
|
||||
() -> coordinator.areSegmentsLoaded(baseName + "-" + datasourceNumber),
|
||||
true,
|
||||
10000,
|
||||
10,
|
||||
"Kafka segments loaded"
|
||||
);
|
||||
}
|
||||
|
||||
queryHelper.testQueriesFromString(queryResponseTemplate);
|
||||
|
||||
for (int i = 0; i < numDatasources; i++) {
|
||||
final String datasource = baseName + "-" + i;
|
||||
List<String> intervals = coordinator.getSegmentIntervals(datasource);
|
||||
|
||||
Collections.sort(intervals);
|
||||
String first = intervals.get(0).split("/")[0];
|
||||
String last = intervals.get(intervals.size() - 1).split("/")[1];
|
||||
Interval interval = Intervals.of(first + "/" + last);
|
||||
coordinator.unloadSegmentsForDataSource(baseName + "-" + i);
|
||||
ITRetryUtil.retryUntilFalse(
|
||||
() -> coordinator.areSegmentsLoaded(datasource),
|
||||
"Segment Unloading"
|
||||
);
|
||||
coordinator.deleteSegmentsDataSource(baseName + "-" + i, interval);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* sad version of
|
||||
* {@link org.apache.druid.tests.indexer.AbstractKafkaIndexingServiceTest#generateStreamIngestionPropsTransform}
|
||||
*/
|
||||
private Function<String, String> generateStreamIngestionPropsTransform(
|
||||
String streamName,
|
||||
String fullDatasourceName,
|
||||
IntegrationTestingConfig config
|
||||
)
|
||||
{
|
||||
final Map<String, Object> consumerConfigs = KafkaConsumerConfigs.getConsumerProperties();
|
||||
final Properties consumerProperties = new Properties();
|
||||
consumerProperties.putAll(consumerConfigs);
|
||||
consumerProperties.setProperty("bootstrap.servers", config.getKafkaInternalHost());
|
||||
KafkaUtil.addPropertiesFromTestConfig(config, consumerProperties);
|
||||
return spec -> {
|
||||
try {
|
||||
spec = StringUtils.replace(
|
||||
spec,
|
||||
"%%DATASOURCE%%",
|
||||
fullDatasourceName
|
||||
);
|
||||
spec = StringUtils.replace(
|
||||
spec,
|
||||
"%%TOPIC_VALUE%%",
|
||||
streamName
|
||||
);
|
||||
return StringUtils.replace(
|
||||
spec,
|
||||
"%%STREAM_PROPERTIES_VALUE%%",
|
||||
jsonMapper.writeValueAsString(consumerProperties)
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -19,4 +19,4 @@
|
|||
kafka:
|
||||
instances:
|
||||
- port: 9092
|
||||
proxyPort: 9093
|
||||
proxyPort: 9094
|
||||
|
|
|
@ -0,0 +1,199 @@
|
|||
# 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.
|
||||
#-------------------------------------------------------------------------
|
||||
|
||||
# Definition of the query test cluster.
|
||||
# See https://yaml.org/spec/1.2.2 for more about YAML
|
||||
include:
|
||||
- /cluster/Common/zk-metastore.yaml
|
||||
- /cluster/Common/kafka.yaml
|
||||
|
||||
druid:
|
||||
coordinator:
|
||||
instances:
|
||||
- port: 8081
|
||||
overlord:
|
||||
instances:
|
||||
- port: 8090
|
||||
broker:
|
||||
instances:
|
||||
- port: 8082
|
||||
router:
|
||||
instances:
|
||||
- port: 8888
|
||||
historical:
|
||||
instances:
|
||||
- port: 8083
|
||||
middlemanager:
|
||||
instances:
|
||||
- port: 8091
|
||||
|
||||
# Metastore initialization queries.
|
||||
# REPLACE is used so that the statements are idempotent
|
||||
# The fancy formatting is for human consumption, it is compacted internally
|
||||
metastoreInit:
|
||||
- sql: |
|
||||
REPLACE INTO druid_segments (
|
||||
id, dataSource, created_date, start, end, partitioned, version, used, payload
|
||||
)
|
||||
VALUES (
|
||||
'twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9',
|
||||
'twitterstream',
|
||||
'2013-05-13T01:08:18.192Z',
|
||||
'2013-01-01T00:00:00.000Z',
|
||||
'2013-01-02T00:00:00.000Z',
|
||||
0,
|
||||
'2013-01-02T04:13:41.980Z_v9',
|
||||
1,
|
||||
'{"dataSource": "twitterstream",
|
||||
"interval": "2013-01-01T00:00:00.000Z/2013-01-02T00:00:00.000Z",
|
||||
"version": "2013-01-02T04:13:41.980Z_v9",
|
||||
"loadSpec": {
|
||||
"type": "s3_zip",
|
||||
"bucket": "static.druid.io",
|
||||
"key": "data/segments/twitterstream/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/2013-01-02T04:13:41.980Z_v9/0/index.zip"
|
||||
},
|
||||
"dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,
|
||||
rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name",
|
||||
"metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets",
|
||||
"shardSpec": {"type": "none"},
|
||||
"binaryVersion": 9,
|
||||
"size": 445235220,
|
||||
"identifier": "twitterstream_2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z_2013-01-02T04:13:41.980Z_v9"
|
||||
}'
|
||||
)
|
||||
|
||||
- sql: |
|
||||
REPLACE INTO druid_segments (
|
||||
id, dataSource, created_date, start, end, partitioned, version, used, payload
|
||||
)
|
||||
VALUES (
|
||||
'twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9',
|
||||
'twitterstream',
|
||||
'2013-05-13T00:03:28.640Z',
|
||||
'2013-01-02T00:00:00.000Z',
|
||||
'2013-01-03T00:00:00.000Z',
|
||||
0,
|
||||
'2013-01-03T03:44:58.791Z_v9',
|
||||
1,
|
||||
'{"dataSource": "twitterstream",
|
||||
"interval": "2013-01-02T00:00:00.000Z/2013-01-03T00:00:00.000Z",
|
||||
"version": "2013-01-03T03:44:58.791Z_v9",
|
||||
"loadSpec": {
|
||||
"type": "s3_zip",
|
||||
"bucket": "static.druid.io",
|
||||
"key": "data/segments/twitterstream/2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z/2013-01-03T03:44:58.791Z_v9/0/index.zip"
|
||||
},
|
||||
"dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,
|
||||
rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name",
|
||||
"metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets",
|
||||
"shardSpec": {"type": "none"},
|
||||
"binaryVersion": 9,
|
||||
"size": 435325540,
|
||||
"identifier": "twitterstream_2013-01-02T00:00:00.000Z_2013-01-03T00:00:00.000Z_2013-01-03T03:44:58.791Z_v9"
|
||||
}'
|
||||
)
|
||||
|
||||
- sql: |
|
||||
REPLACE INTO druid_segments (
|
||||
id, dataSource, created_date, start, end, partitioned, version, used, payload
|
||||
)
|
||||
VALUES (
|
||||
'twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9',
|
||||
'twitterstream',
|
||||
'2013-05-13T00:03:48.807Z',
|
||||
'2013-01-03T00:00:00.000Z',
|
||||
'2013-01-04T00:00:00.000Z',
|
||||
0,
|
||||
'2013-01-04T04:09:13.590Z_v9',
|
||||
1,
|
||||
'{"dataSource": "twitterstream",
|
||||
"interval": "2013-01-03T00:00:00.000Z/2013-01-04T00:00:00.000Z",
|
||||
"version": "2013-01-04T04:09:13.590Z_v9",
|
||||
"loadSpec": {
|
||||
"type": "s3_zip",
|
||||
"bucket": "static.druid.io",
|
||||
"key": "data/segments/twitterstream/2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z/2013-01-04T04:09:13.590Z_v9/0/index.zip"
|
||||
},
|
||||
"dimensions": "has_links,first_hashtag,user_time_zone,user_location,has_mention,user_lang,
|
||||
rt_name,user_name,is_retweet,is_viral,has_geo,url_domain,user_mention_name,reply_to_name",
|
||||
"metrics": "count,tweet_length,num_followers,num_links,num_mentions,num_hashtags,num_favorites,user_total_tweets",
|
||||
"shardSpec": {"type": "none"},
|
||||
"binaryVersion": 9,
|
||||
"size": 411651320,
|
||||
"identifier": "twitterstream_2013-01-03T00:00:00.000Z_2013-01-04T00:00:00.000Z_2013-01-04T04:09:13.590Z_v9"
|
||||
}'
|
||||
)
|
||||
|
||||
- sql: |
|
||||
REPLACE INTO druid_segments (
|
||||
id, dataSource, created_date, start, end, partitioned, version, used, payload
|
||||
)
|
||||
VALUES (
|
||||
'wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9',
|
||||
'wikipedia_editstream',
|
||||
'2013-03-15T20:49:52.348Z',
|
||||
'2012-12-29T00:00:00.000Z',
|
||||
'2013-01-10T08:00:00.000Z',
|
||||
0,
|
||||
'2013-01-10T08:13:47.830Z_v9',
|
||||
1,
|
||||
'{"dataSource": "wikipedia_editstream",
|
||||
"interval": "2012-12-29T00:00:00.000Z/2013-01-10T08:00:00.000Z",
|
||||
"version": "2013-01-10T08:13:47.830Z_v9",
|
||||
"loadSpec": {
|
||||
"type": "s3_zip",
|
||||
"bucket": "static.druid.io",
|
||||
"key": "data/segments/wikipedia_editstream/2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z/2013-01-10T08:13:47.830Z_v9/0/index.zip"},
|
||||
"dimensions": "anonymous,area_code,city,continent_code,country_name,dma_code,geo,language,
|
||||
namespace,network,newpage,page,postal_code,region_lookup,robot,unpatrolled,user",
|
||||
"metrics": "added,count,deleted,delta,delta_hist,unique_users,variation",
|
||||
"shardSpec": {"type": "none"},
|
||||
"binaryVersion": 9,
|
||||
"size": 446027801,
|
||||
"identifier": "wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9"
|
||||
}'
|
||||
)
|
||||
|
||||
- sql: |
|
||||
REPLACE INTO druid_segments (
|
||||
id, dataSource, created_date, start, end, partitioned, version, used, payload
|
||||
)
|
||||
VALUES (
|
||||
'wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z',
|
||||
'wikipedia',
|
||||
'2013-08-08T21:26:23.799Z',
|
||||
'2013-08-01T00:00:00.000Z',
|
||||
'2013-08-02T00:00:00.000Z',
|
||||
0,
|
||||
'2013-08-08T21:22:48.989Z',
|
||||
1,
|
||||
'{"dataSource": "wikipedia",
|
||||
"interval": "2013-08-01T00:00:00.000Z/2013-08-02T00:00:00.000Z",
|
||||
"version": "2013-08-08T21:22:48.989Z",
|
||||
"loadSpec": {
|
||||
"type": "s3_zip",
|
||||
"bucket": "static.druid.io",
|
||||
"key": "data/segments/wikipedia/20130801T000000.000Z_20130802T000000.000Z/2013-08-08T21_22_48.989Z/0/index.zip"
|
||||
},
|
||||
"dimensions": "dma_code,continent_code,geo,area_code,robot,country_name,network,city,namespace,
|
||||
anonymous,unpatrolled,page,postal_code,language,newpage,user,region_lookup",
|
||||
"metrics": "count,delta,variation,added,deleted",
|
||||
"shardSpec": {"type": "none"},
|
||||
"binaryVersion": 9,
|
||||
"size": 24664730,
|
||||
"identifier": "wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z"
|
||||
}'
|
||||
)
|
|
@ -61,7 +61,7 @@
|
|||
"type": "index_parallel",
|
||||
"inputSource": {
|
||||
"type": "local",
|
||||
"baseDir": "/resources/data/union_query/",
|
||||
"baseDir": "/resources/data/broadcast/",
|
||||
"filter": "wikipedia_index_data*"
|
||||
},
|
||||
"appendToExisting": false,
|
||||
|
|
|
@ -1,94 +0,0 @@
|
|||
{
|
||||
"type": "index_realtime_appenderator",
|
||||
"spec": {
|
||||
"dataSchema": {
|
||||
"dataSource": "%%DATASOURCE%%",
|
||||
"metricsSpec": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "count"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "added",
|
||||
"fieldName": "added"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "deleted",
|
||||
"fieldName": "deleted"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "delta",
|
||||
"fieldName": "delta"
|
||||
}
|
||||
],
|
||||
"granularitySpec": {
|
||||
"segmentGranularity": "minute",
|
||||
"queryGranularity": "second"
|
||||
},
|
||||
"parser": {
|
||||
"type": "map",
|
||||
"parseSpec": {
|
||||
"format": "tsv",
|
||||
"columns": [
|
||||
"timestamp",
|
||||
"page",
|
||||
"language",
|
||||
"user",
|
||||
"unpatrolled",
|
||||
"newPage",
|
||||
"robot",
|
||||
"anonymous",
|
||||
"namespace",
|
||||
"continent",
|
||||
"country",
|
||||
"region",
|
||||
"city",
|
||||
"added",
|
||||
"deleted",
|
||||
"delta"
|
||||
],
|
||||
"timestampSpec": {
|
||||
"column": "timestamp",
|
||||
"format": "iso"
|
||||
},
|
||||
"dimensionsSpec": {
|
||||
"dimensions": [
|
||||
"page",
|
||||
"language",
|
||||
"user",
|
||||
"unpatrolled",
|
||||
"newPage",
|
||||
"robot",
|
||||
"anonymous",
|
||||
"namespace",
|
||||
"continent",
|
||||
"country",
|
||||
"region",
|
||||
"city"
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"ioConfig": {
|
||||
"type": "realtime",
|
||||
"firehose": {
|
||||
"type": "timed",
|
||||
"shutoffTime": "#SHUTOFFTIME",
|
||||
"delegate": {
|
||||
"type": "receiver",
|
||||
"serviceName": "eventReceiverServiceName",
|
||||
"bufferSize": 100000
|
||||
}
|
||||
}
|
||||
},
|
||||
"tuningConfig": {
|
||||
"type": "realtime_appenderator",
|
||||
"maxRowsInMemory": 1,
|
||||
"intermediatePersistPeriod": "PT1M"
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,98 +0,0 @@
|
|||
{
|
||||
"type": "index_realtime",
|
||||
"spec": {
|
||||
"dataSchema": {
|
||||
"dataSource": "%%DATASOURCE%%",
|
||||
"metricsSpec": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "count"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "added",
|
||||
"fieldName": "added"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "deleted",
|
||||
"fieldName": "deleted"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "delta",
|
||||
"fieldName": "delta"
|
||||
}
|
||||
],
|
||||
"granularitySpec": {
|
||||
"segmentGranularity": "minute",
|
||||
"queryGranularity": "second"
|
||||
},
|
||||
"parser": {
|
||||
"type": "map",
|
||||
"parseSpec": {
|
||||
"format": "tsv",
|
||||
"columns": [
|
||||
"timestamp",
|
||||
"page",
|
||||
"language",
|
||||
"user",
|
||||
"unpatrolled",
|
||||
"newPage",
|
||||
"robot",
|
||||
"anonymous",
|
||||
"namespace",
|
||||
"continent",
|
||||
"country",
|
||||
"region",
|
||||
"city",
|
||||
"added",
|
||||
"deleted",
|
||||
"delta"
|
||||
],
|
||||
"timestampSpec": {
|
||||
"column": "timestamp",
|
||||
"format": "iso"
|
||||
},
|
||||
"dimensionsSpec": {
|
||||
"dimensions": [
|
||||
"page",
|
||||
"language",
|
||||
"user",
|
||||
"unpatrolled",
|
||||
"newPage",
|
||||
"robot",
|
||||
"anonymous",
|
||||
"namespace",
|
||||
"continent",
|
||||
"country",
|
||||
"region",
|
||||
"city"
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"ioConfig": {
|
||||
"type": "realtime",
|
||||
"firehose": {
|
||||
"type": "timed",
|
||||
"shutoffTime": "#SHUTOFFTIME",
|
||||
"delegate": {
|
||||
"type": "receiver",
|
||||
"serviceName": "eventReceiverServiceName",
|
||||
"bufferSize": 100000
|
||||
}
|
||||
}
|
||||
},
|
||||
"tuningConfig": {
|
||||
"type": "realtime",
|
||||
"maxRowsInMemory": 1,
|
||||
"intermediatePersistPeriod": "PT1M",
|
||||
"windowPeriod": "PT1M",
|
||||
"rejectionPolicy": {
|
||||
"type": "serverTime"
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,98 +0,0 @@
|
|||
{
|
||||
"type": "index_realtime",
|
||||
"spec": {
|
||||
"dataSchema": {
|
||||
"dataSource": "%%DATASOURCE%%",
|
||||
"metricsSpec": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "count"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "added",
|
||||
"fieldName": "added"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "deleted",
|
||||
"fieldName": "deleted"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "delta",
|
||||
"fieldName": "delta"
|
||||
}
|
||||
],
|
||||
"granularitySpec": {
|
||||
"segmentGranularity": "DAY",
|
||||
"queryGranularity": "second"
|
||||
},
|
||||
"parser": {
|
||||
"type": "map",
|
||||
"parseSpec": {
|
||||
"format": "tsv",
|
||||
"columns": [
|
||||
"timestamp",
|
||||
"page",
|
||||
"language",
|
||||
"user",
|
||||
"unpatrolled",
|
||||
"newPage",
|
||||
"robot",
|
||||
"anonymous",
|
||||
"namespace",
|
||||
"continent",
|
||||
"country",
|
||||
"region",
|
||||
"city",
|
||||
"added",
|
||||
"deleted",
|
||||
"delta"
|
||||
],
|
||||
"timestampSpec": {
|
||||
"column": "timestamp",
|
||||
"format": "iso"
|
||||
},
|
||||
"dimensionsSpec": {
|
||||
"dimensions": [
|
||||
"page",
|
||||
"language",
|
||||
"user",
|
||||
"unpatrolled",
|
||||
"newPage",
|
||||
"robot",
|
||||
"anonymous",
|
||||
"namespace",
|
||||
"continent",
|
||||
"country",
|
||||
"region",
|
||||
"city"
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"ioConfig": {
|
||||
"type": "realtime",
|
||||
"firehose": {
|
||||
"type": "timed",
|
||||
"shutoffTime": "#SHUTOFFTIME",
|
||||
"delegate": {
|
||||
"type": "receiver",
|
||||
"serviceName": "eventReceiverServiceName",
|
||||
"bufferSize": 100000
|
||||
}
|
||||
}
|
||||
},
|
||||
"tuningConfig": {
|
||||
"type": "realtime",
|
||||
"maxRowsInMemory": 1,
|
||||
"intermediatePersistPeriod": "PT1M",
|
||||
"windowPeriod": "PT1M",
|
||||
"rejectionPolicy": {
|
||||
"type": "none"
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,69 @@
|
|||
{
|
||||
"type": "kafka",
|
||||
"dataSchema": {
|
||||
"dataSource": "%%DATASOURCE%%",
|
||||
"timestampSpec": {
|
||||
"column": "timestamp",
|
||||
"format": "auto"
|
||||
},
|
||||
"dimensionsSpec": {
|
||||
"dimensions": [
|
||||
"page",
|
||||
"language",
|
||||
"user",
|
||||
"unpatrolled",
|
||||
"newPage",
|
||||
"robot",
|
||||
"anonymous",
|
||||
"namespace",
|
||||
"continent",
|
||||
"country",
|
||||
"region",
|
||||
"city"
|
||||
],
|
||||
"dimensionExclusions": [],
|
||||
"spatialDimensions": []
|
||||
},
|
||||
"metricsSpec": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "count"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "added",
|
||||
"fieldName": "added"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "deleted",
|
||||
"fieldName": "deleted"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "delta",
|
||||
"fieldName": "delta"
|
||||
}
|
||||
],
|
||||
"granularitySpec": {
|
||||
"type": "uniform",
|
||||
"segmentGranularity": "DAY",
|
||||
"queryGranularity": "second"
|
||||
}
|
||||
},
|
||||
"tuningConfig": {
|
||||
"type": "kafka",
|
||||
"intermediatePersistPeriod": "PT1H",
|
||||
"maxRowsPerSegment": 5000000,
|
||||
"maxRowsInMemory": 500000
|
||||
},
|
||||
"ioConfig": {
|
||||
"topic": "%%TOPIC_VALUE%%",
|
||||
"consumerProperties": %%STREAM_PROPERTIES_VALUE%%,
|
||||
"taskCount": 2,
|
||||
"replicas": 1,
|
||||
"taskDuration": "PT120S",
|
||||
"useEarliestOffset": true,
|
||||
"inputFormat" : {"type": "json"}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,566 @@
|
|||
[
|
||||
{
|
||||
"description": "timeseries, filtered, all aggs, all",
|
||||
"query": {
|
||||
"queryType": "timeseries",
|
||||
"dataSource": {
|
||||
"type": "union",
|
||||
"dataSources": [
|
||||
"%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3",
|
||||
"%%DATASOURCE%%-0"
|
||||
]
|
||||
},
|
||||
"intervals": ["2013-08-31/2013-09-01"],
|
||||
"granularity": "all",
|
||||
"filter": {
|
||||
"type": "selector",
|
||||
"dimension": "language",
|
||||
"value": "en"
|
||||
},
|
||||
"aggregations": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "rows"
|
||||
},
|
||||
{
|
||||
"type": "longSum",
|
||||
"fieldName": "count",
|
||||
"name": "count"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "added",
|
||||
"name": "added"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "deleted",
|
||||
"name": "deleted"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "delta",
|
||||
"name": "delta"
|
||||
}
|
||||
],
|
||||
"context": {
|
||||
"useCache": "true",
|
||||
"populateCache": "true",
|
||||
"timeout": 60000
|
||||
}
|
||||
},
|
||||
"expectedResults": [
|
||||
{
|
||||
"timestamp": "2013-08-31T01:02:33.000Z",
|
||||
"result": {
|
||||
"added": 1548.0,
|
||||
"count": 6,
|
||||
"delta": 561.0,
|
||||
"deleted": 987.0,
|
||||
"rows": 6
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"description": "topN, all aggs, page dim, uniques metric",
|
||||
"query": {
|
||||
"queryType": "topN",
|
||||
"dataSource": {
|
||||
"type": "union",
|
||||
"dataSources": [
|
||||
"%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3",
|
||||
"%%DATASOURCE%%-0"
|
||||
]
|
||||
},
|
||||
"intervals": ["2013-08-31/2013-09-01"],
|
||||
"granularity": "all",
|
||||
"aggregations": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "rows"
|
||||
},
|
||||
{
|
||||
"type": "longSum",
|
||||
"fieldName": "count",
|
||||
"name": "count"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "added",
|
||||
"name": "added"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "deleted",
|
||||
"name": "deleted"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "delta",
|
||||
"name": "delta"
|
||||
}
|
||||
],
|
||||
"dimension": "page",
|
||||
"metric": "added",
|
||||
"threshold": 3,
|
||||
"context": {
|
||||
"useCache": "true",
|
||||
"populateCache": "true",
|
||||
"timeout": 60000
|
||||
}
|
||||
},
|
||||
"expectedResults": [
|
||||
{
|
||||
"timestamp": "2013-08-31T01:02:33.000Z",
|
||||
"result": [
|
||||
{
|
||||
"added": 2715.0,
|
||||
"count": 3,
|
||||
"page": "Crimson Typhoon",
|
||||
"delta": 2700.0,
|
||||
"deleted": 15.0,
|
||||
"rows": 3
|
||||
},
|
||||
{
|
||||
"added": 1377.0,
|
||||
"count": 3,
|
||||
"page": "Striker Eureka",
|
||||
"delta": 990.0,
|
||||
"deleted": 387.0,
|
||||
"rows": 3
|
||||
},
|
||||
{
|
||||
"added": 369.0,
|
||||
"count": 3,
|
||||
"page": "Cherno Alpha",
|
||||
"delta": 333.0,
|
||||
"deleted": 36.0,
|
||||
"rows": 3
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"description": "topN, all aggs, page dim, count metric, postAggs",
|
||||
"query": {
|
||||
"queryType": "topN",
|
||||
"dataSource": {
|
||||
"type": "union",
|
||||
"dataSources": [
|
||||
"%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3",
|
||||
"%%DATASOURCE%%-0"
|
||||
]
|
||||
},
|
||||
"intervals": ["2013-08-31/2013-09-01"],
|
||||
"granularity": "all",
|
||||
"aggregations": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "rows"
|
||||
},
|
||||
{
|
||||
"type": "longSum",
|
||||
"fieldName": "count",
|
||||
"name": "count"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "added",
|
||||
"name": "added"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "deleted",
|
||||
"name": "deleted"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "delta",
|
||||
"name": "delta"
|
||||
}
|
||||
],
|
||||
"postAggregations": [
|
||||
{
|
||||
"type": "arithmetic",
|
||||
"name": "sumOfAddedDeletedConst",
|
||||
"fn": "+",
|
||||
"fields": [
|
||||
{
|
||||
"type": "fieldAccess",
|
||||
"name": "added",
|
||||
"fieldName": "added"
|
||||
},
|
||||
{
|
||||
"type": "arithmetic",
|
||||
"name": "",
|
||||
"fn": "+",
|
||||
"fields": [
|
||||
{
|
||||
"type": "fieldAccess",
|
||||
"name": "deleted",
|
||||
"fieldName": "deleted"
|
||||
},
|
||||
{
|
||||
"type": "constant",
|
||||
"name": "constant",
|
||||
"value": 1000
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
],
|
||||
"dimension": "page",
|
||||
"metric": "added",
|
||||
"threshold": 3,
|
||||
"context": {
|
||||
"useCache": "true",
|
||||
"populateCache": "true",
|
||||
"timeout": 60000
|
||||
}
|
||||
},
|
||||
"expectedResults": [
|
||||
{
|
||||
"timestamp": "2013-08-31T01:02:33.000Z",
|
||||
"result": [
|
||||
{
|
||||
"added": 2715.0,
|
||||
"count": 3,
|
||||
"page": "Crimson Typhoon",
|
||||
"delta": 2700.0,
|
||||
"deleted": 15.0,
|
||||
"sumOfAddedDeletedConst": 3730.0,
|
||||
"rows": 3
|
||||
},
|
||||
{
|
||||
"added": 1377.0,
|
||||
"count": 3,
|
||||
"page": "Striker Eureka",
|
||||
"delta": 990.0,
|
||||
"deleted": 387.0,
|
||||
"sumOfAddedDeletedConst": 2764.0,
|
||||
"rows": 3
|
||||
},
|
||||
{
|
||||
"added": 369.0,
|
||||
"count": 3,
|
||||
"page": "Cherno Alpha",
|
||||
"delta": 333.0,
|
||||
"deleted": 36.0,
|
||||
"sumOfAddedDeletedConst": 1405.0,
|
||||
"rows": 3
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"description": "topN, lexicographic, two aggs, language dim, postAggs",
|
||||
"query": {
|
||||
"queryType": "topN",
|
||||
"dataSource": {
|
||||
"type": "union",
|
||||
"dataSources": [
|
||||
"%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3",
|
||||
"%%DATASOURCE%%-0"
|
||||
]
|
||||
},
|
||||
"intervals": ["2013-08-31/2013-09-01"],
|
||||
"granularity": "all",
|
||||
"aggregations": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "rows"
|
||||
},
|
||||
{
|
||||
"type": "longSum",
|
||||
"fieldName": "count",
|
||||
"name": "count"
|
||||
}
|
||||
],
|
||||
"postAggregations": [
|
||||
{
|
||||
"type": "arithmetic",
|
||||
"name": "sumOfRowsAndCount",
|
||||
"fn": "+",
|
||||
"fields": [
|
||||
{
|
||||
"type": "fieldAccess",
|
||||
"name": "rows",
|
||||
"fieldName": "rows"
|
||||
},
|
||||
{
|
||||
"type": "fieldAccess",
|
||||
"name": "count",
|
||||
"fieldName": "count"
|
||||
}
|
||||
]
|
||||
}
|
||||
],
|
||||
"dimension": "language",
|
||||
"metric": {
|
||||
"type": "lexicographic",
|
||||
"previousStop": "a"
|
||||
},
|
||||
"threshold": 3,
|
||||
"context": {
|
||||
"useCache": "true",
|
||||
"populateCache": "true",
|
||||
"timeout": 60000
|
||||
}
|
||||
},
|
||||
"expectedResults": [
|
||||
{
|
||||
"timestamp": "2013-08-31T01:02:33.000Z",
|
||||
"result": [
|
||||
{
|
||||
"sumOfRowsAndCount": 12.0,
|
||||
"count": 6,
|
||||
"language": "en",
|
||||
"rows": 6
|
||||
},
|
||||
{
|
||||
"sumOfRowsAndCount": 6.0,
|
||||
"count": 3,
|
||||
"language": "ja",
|
||||
"rows": 3
|
||||
},
|
||||
{
|
||||
"sumOfRowsAndCount": 6.0,
|
||||
"count": 3,
|
||||
"language": "ru",
|
||||
"rows": 3
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"description": "groupBy, two aggs, namespace dim, postAggs",
|
||||
"query": {
|
||||
"queryType": "groupBy",
|
||||
"dataSource": {
|
||||
"type": "union",
|
||||
"dataSources": [
|
||||
"%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3",
|
||||
"%%DATASOURCE%%-0"
|
||||
]
|
||||
},
|
||||
"intervals": ["2013-08-31/2013-09-01"],
|
||||
"granularity": "all",
|
||||
"aggregations": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "rows"
|
||||
},
|
||||
{
|
||||
"type": "longSum",
|
||||
"fieldName": "count",
|
||||
"name": "count"
|
||||
}
|
||||
],
|
||||
"postAggregations": [
|
||||
{
|
||||
"type": "arithmetic",
|
||||
"name": "sumOfRowsAndCount",
|
||||
"fn": "+",
|
||||
"fields": [
|
||||
{
|
||||
"type": "fieldAccess",
|
||||
"name": "rows",
|
||||
"fieldName": "rows"
|
||||
},
|
||||
{
|
||||
"type": "fieldAccess",
|
||||
"name": "count",
|
||||
"fieldName": "count"
|
||||
}
|
||||
]
|
||||
}
|
||||
],
|
||||
"dimensions": ["namespace"],
|
||||
"context": {
|
||||
"useCache": "true",
|
||||
"populateCache": "true",
|
||||
"timeout": 60000
|
||||
}
|
||||
},
|
||||
"expectedResults": [
|
||||
{
|
||||
"version": "v1",
|
||||
"timestamp": "2013-08-31T00:00:00.000Z",
|
||||
"event": {
|
||||
"sumOfRowsAndCount": 12.0,
|
||||
"count": 6,
|
||||
"rows": 6,
|
||||
"namespace": "article"
|
||||
}
|
||||
},
|
||||
{
|
||||
"version": "v1",
|
||||
"timestamp": "2013-08-31T00:00:00.000Z",
|
||||
"event": {
|
||||
"sumOfRowsAndCount": 18.0,
|
||||
"count": 9,
|
||||
"rows": 9,
|
||||
"namespace": "wikipedia"
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"description": "groupBy, two aggs, namespace + robot dim, postAggs",
|
||||
"query": {
|
||||
"queryType": "groupBy",
|
||||
"dataSource": {
|
||||
"type": "union",
|
||||
"dataSources": [
|
||||
"%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3",
|
||||
"%%DATASOURCE%%-0"
|
||||
]
|
||||
},
|
||||
"intervals": ["2013-08-31/2013-09-01"],
|
||||
"granularity": "all",
|
||||
"aggregations": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "rows"
|
||||
},
|
||||
{
|
||||
"type": "longSum",
|
||||
"fieldName": "count",
|
||||
"name": "count"
|
||||
}
|
||||
],
|
||||
"postAggregations": [
|
||||
{
|
||||
"type": "arithmetic",
|
||||
"name": "sumOfRowsAndCount",
|
||||
"fn": "+",
|
||||
"fields": [
|
||||
{
|
||||
"type": "fieldAccess",
|
||||
"name": "rows",
|
||||
"fieldName": "rows"
|
||||
},
|
||||
{
|
||||
"type": "fieldAccess",
|
||||
"name": "count",
|
||||
"fieldName": "count"
|
||||
}
|
||||
]
|
||||
}
|
||||
],
|
||||
"dimensions": ["namespace", "robot"],
|
||||
"limitSpec": {
|
||||
"type": "default",
|
||||
"limit": 3,
|
||||
"orderBy": ["robot", "namespace"]
|
||||
},
|
||||
"context": {
|
||||
"useCache": "true",
|
||||
"populateCache": "true",
|
||||
"timeout": 60000
|
||||
}
|
||||
},
|
||||
"expectedResults": [
|
||||
{
|
||||
"version": "v1",
|
||||
"timestamp": "2013-08-31T00:00:00.000Z",
|
||||
"event": {
|
||||
"sumOfRowsAndCount": 6.0,
|
||||
"count": 3,
|
||||
"robot": "false",
|
||||
"rows": 3,
|
||||
"namespace": "article"
|
||||
}
|
||||
},
|
||||
{
|
||||
"version": "v1",
|
||||
"timestamp": "2013-08-31T00:00:00.000Z",
|
||||
"event": {
|
||||
"sumOfRowsAndCount": 6.0,
|
||||
"count": 3,
|
||||
"robot": "true",
|
||||
"rows": 3,
|
||||
"namespace": "article"
|
||||
}
|
||||
},
|
||||
{
|
||||
"version": "v1",
|
||||
"timestamp": "2013-08-31T00:00:00.000Z",
|
||||
"event": {
|
||||
"sumOfRowsAndCount": 18.0,
|
||||
"count": 9,
|
||||
"robot": "true",
|
||||
"rows": 9,
|
||||
"namespace": "wikipedia"
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"query": {
|
||||
"queryType": "search",
|
||||
"intervals": ["2013-08-31/2013-09-01"],
|
||||
"dataSource": {
|
||||
"type": "union",
|
||||
"dataSources": [
|
||||
"%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3",
|
||||
"%%DATASOURCE%%-0"
|
||||
]
|
||||
},
|
||||
"granularity": "all",
|
||||
"query": {
|
||||
"type": "insensitive_contains",
|
||||
"value": "ip"
|
||||
},
|
||||
"context": {
|
||||
"useCache": "true",
|
||||
"populateCache": "true",
|
||||
"timeout": 60000
|
||||
}
|
||||
},
|
||||
"expectedResults": [
|
||||
{
|
||||
"timestamp": "2013-08-31T00:00:00.000Z",
|
||||
"result": [
|
||||
{
|
||||
"dimension": "user",
|
||||
"value": "triplets",
|
||||
"count":3
|
||||
},
|
||||
{
|
||||
"dimension": "namespace",
|
||||
"value": "wikipedia",
|
||||
"count":9
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"description": "timeboundary, 1 agg, union",
|
||||
"query": {
|
||||
"queryType": "timeBoundary",
|
||||
"dataSource": {
|
||||
"type": "union",
|
||||
"dataSources": [
|
||||
"%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3",
|
||||
"%%DATASOURCE%%-0"
|
||||
]
|
||||
}
|
||||
},
|
||||
"expectedResults": [
|
||||
{
|
||||
"timestamp": "2013-08-31T01:02:33.000Z",
|
||||
"result": {
|
||||
"minTime": "2013-08-31T01:02:33.000Z",
|
||||
"maxTime": "2013-09-01T12:41:27.000Z"
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
|
@ -58,10 +58,6 @@ public class TestNGGroup
|
|||
|
||||
public static final String QUERY_ERROR = "query-error";
|
||||
|
||||
public static final String CLI_INDEXER = "cli-indexer";
|
||||
|
||||
public static final String REALTIME_INDEX = "realtime-index";
|
||||
|
||||
/**
|
||||
* This group can only be run individually using -Dgroups=security since it requires specific test data setup.
|
||||
*/
|
||||
|
|
|
@ -1,198 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.tests.indexer;
|
||||
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.druid.curator.discovery.ServerDiscoveryFactory;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.java.util.http.client.HttpClient;
|
||||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import org.apache.druid.testing.IntegrationTestingConfig;
|
||||
import org.apache.druid.testing.guice.TestClient;
|
||||
import org.apache.druid.testing.utils.ITRetryUtil;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.format.DateTimeFormat;
|
||||
import org.joda.time.format.DateTimeFormatter;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.InputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* Steps
|
||||
* 1) Submit a realtime index task
|
||||
* 2) Load data using EventReceiverFirehose
|
||||
* 3) Run queries to verify that the ingested data is available for queries
|
||||
* 4) Wait for handover of the segment to historical node
|
||||
* 5) Query data (will be from historical node)
|
||||
* 6) Disable and delete the created data segment
|
||||
*/
|
||||
public abstract class AbstractITRealtimeIndexTaskTest extends AbstractIndexerTest
|
||||
{
|
||||
static final String EVENT_RECEIVER_SERVICE_NAME = "eventReceiverServiceName";
|
||||
static final String EVENT_DATA_FILE = "/indexer/wikipedia_realtime_index_data.json";
|
||||
|
||||
private static final Logger LOG = new Logger(AbstractITRealtimeIndexTaskTest.class);
|
||||
private static final String INDEX_DATASOURCE = "wikipedia_index_test";
|
||||
|
||||
static final int DELAY_BETWEEN_EVENTS_SECS = 4;
|
||||
final String TIME_PLACEHOLDER = "YYYY-MM-DDTHH:MM:SS";
|
||||
// format for putting timestamp into events
|
||||
static final DateTimeFormatter EVENT_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss");
|
||||
// format for the querying interval
|
||||
private static final DateTimeFormatter INTERVAL_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:'00Z'");
|
||||
// format for the expected timestamp in a query response
|
||||
private static final DateTimeFormatter TIMESTAMP_FMT = DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss'.000Z'");
|
||||
DateTime dtFirst; // timestamp of 1st event
|
||||
DateTime dtLast; // timestamp of last event
|
||||
DateTime dtGroupBy; // timestamp for expected response for groupBy query
|
||||
|
||||
static final int NUM_RETRIES = 60;
|
||||
static final long DELAY_FOR_RETRIES_MS = 10000;
|
||||
|
||||
@Inject
|
||||
ServerDiscoveryFactory factory;
|
||||
@Inject
|
||||
@TestClient
|
||||
HttpClient httpClient;
|
||||
|
||||
@Inject
|
||||
IntegrationTestingConfig config;
|
||||
|
||||
private String fullDatasourceName;
|
||||
|
||||
void doTest()
|
||||
{
|
||||
fullDatasourceName = INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix();
|
||||
|
||||
LOG.info("Starting test: %s", this.getClass().getSimpleName());
|
||||
try (final Closeable ignored = unloader(fullDatasourceName)) {
|
||||
// the task will run for 5 minutes and then shutdown itself
|
||||
String task = setShutOffTime(
|
||||
getResourceAsString(getTaskResource()),
|
||||
DateTimes.utc(System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(5))
|
||||
);
|
||||
task = StringUtils.replace(task, "%%DATASOURCE%%", fullDatasourceName);
|
||||
|
||||
LOG.info("indexerSpec: [%s]\n", task);
|
||||
String taskID = indexer.submitTask(task);
|
||||
|
||||
// sleep for a while to let peons finish starting up
|
||||
TimeUnit.SECONDS.sleep(60);
|
||||
|
||||
// this posts 22 events, one every 4 seconds
|
||||
// each event contains the current time as its timestamp except
|
||||
// the timestamp for the 14th event is early enough that the event should be ignored
|
||||
// the timestamp for the 18th event is 2 seconds earlier than the 17th
|
||||
postEvents();
|
||||
|
||||
// wait for a while to let the events be ingested
|
||||
ITRetryUtil.retryUntil(
|
||||
() -> {
|
||||
final int countRows = queryHelper.countRows(
|
||||
fullDatasourceName,
|
||||
Intervals.ETERNITY,
|
||||
name -> new LongSumAggregatorFactory(name, "count")
|
||||
);
|
||||
return countRows == getNumExpectedRowsIngested();
|
||||
},
|
||||
true,
|
||||
DELAY_FOR_RETRIES_MS,
|
||||
NUM_RETRIES,
|
||||
"Waiting all events are ingested"
|
||||
);
|
||||
|
||||
// put the timestamps into the query structure
|
||||
String query_response_template;
|
||||
InputStream is = ITRealtimeIndexTaskTest.class.getResourceAsStream(getQueriesResource());
|
||||
if (null == is) {
|
||||
throw new ISE("could not open query file: %s", getQueriesResource());
|
||||
}
|
||||
query_response_template = IOUtils.toString(is, StandardCharsets.UTF_8);
|
||||
|
||||
String queryStr = query_response_template;
|
||||
queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst));
|
||||
queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", TIMESTAMP_FMT.print(dtLast));
|
||||
queryStr = StringUtils.replace(queryStr, "%%TIMEBOUNDARY_RESPONSE_MINTIME%%", TIMESTAMP_FMT.print(dtFirst));
|
||||
queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_QUERY_START%%", INTERVAL_FMT.print(dtFirst));
|
||||
queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_QUERY_END%%", INTERVAL_FMT.print(dtLast.plusMinutes(2)));
|
||||
queryStr = StringUtils.replace(queryStr, "%%TIMESERIES_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst));
|
||||
queryStr = StringUtils.replace(queryStr, "%%POST_AG_REQUEST_START%%", INTERVAL_FMT.print(dtFirst));
|
||||
queryStr = StringUtils.replace(queryStr, "%%POST_AG_REQUEST_END%%", INTERVAL_FMT.print(dtLast.plusMinutes(2)));
|
||||
String postAgResponseTimestamp = TIMESTAMP_FMT.print(dtGroupBy.withSecondOfMinute(0));
|
||||
queryStr = StringUtils.replace(queryStr, "%%POST_AG_RESPONSE_TIMESTAMP%%", postAgResponseTimestamp);
|
||||
queryStr = StringUtils.replace(queryStr, "%%DATASOURCE%%", fullDatasourceName);
|
||||
|
||||
// should hit the queries all on realtime task or some on realtime task
|
||||
// and some on historical. Which it is depends on where in the minute we were
|
||||
// when we started posting events.
|
||||
try {
|
||||
this.queryHelper.testQueriesFromString(getRouterURL(), queryStr);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
// wait for the task to complete
|
||||
indexer.waitUntilTaskCompletes(taskID);
|
||||
|
||||
// task should complete only after the segments are loaded by historical node
|
||||
ITRetryUtil.retryUntil(
|
||||
() -> coordinator.areSegmentsLoaded(fullDatasourceName),
|
||||
true,
|
||||
DELAY_FOR_RETRIES_MS,
|
||||
NUM_RETRIES,
|
||||
"Real-time generated segments loaded"
|
||||
);
|
||||
|
||||
// queries should be answered by historical
|
||||
this.queryHelper.testQueriesFromString(getRouterURL(), queryStr);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private String setShutOffTime(String taskAsString, DateTime time)
|
||||
{
|
||||
return StringUtils.replace(taskAsString, "#SHUTOFFTIME", time.toString());
|
||||
}
|
||||
|
||||
private String getRouterURL()
|
||||
{
|
||||
return StringUtils.format(
|
||||
"%s/druid/v2?pretty",
|
||||
config.getRouterUrl()
|
||||
);
|
||||
}
|
||||
|
||||
abstract String getTaskResource();
|
||||
abstract String getQueriesResource();
|
||||
|
||||
abstract void postEvents() throws Exception;
|
||||
|
||||
abstract int getNumExpectedRowsIngested();
|
||||
}
|
|
@ -171,7 +171,7 @@ public abstract class AbstractIndexerTest
|
|||
|
||||
public static InputStream getResourceAsStream(String resource)
|
||||
{
|
||||
return ITRealtimeIndexTaskTest.class.getResourceAsStream(resource);
|
||||
return ITCompactionTaskTest.class.getResourceAsStream(resource);
|
||||
}
|
||||
|
||||
public static List<String> listResources(String dir) throws IOException
|
||||
|
|
|
@ -1,153 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.tests.indexer;
|
||||
|
||||
import org.apache.druid.curator.discovery.ServerDiscoverySelector;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
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.testing.clients.EventReceiverFirehoseTestClient;
|
||||
import org.apache.druid.testing.guice.DruidTestModuleFactory;
|
||||
import org.apache.druid.testing.utils.ServerDiscoveryUtil;
|
||||
import org.apache.druid.tests.TestNGGroup;
|
||||
import org.joda.time.DateTime;
|
||||
import org.testng.annotations.Guice;
|
||||
import org.testng.annotations.Test;
|
||||
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import java.io.BufferedReader;
|
||||
import java.io.InputStreamReader;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* See {@link AbstractITRealtimeIndexTaskTest} for test details.
|
||||
*/
|
||||
@Test(groups = {TestNGGroup.REALTIME_INDEX, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_METADATA_QUERY_DISABLED})
|
||||
@Guice(moduleFactory = DruidTestModuleFactory.class)
|
||||
public class ITAppenderatorDriverRealtimeIndexTaskTest extends AbstractITRealtimeIndexTaskTest
|
||||
{
|
||||
private static final Logger LOG = new Logger(ITAppenderatorDriverRealtimeIndexTaskTest.class);
|
||||
private static final String REALTIME_TASK_RESOURCE = "/indexer/wikipedia_realtime_appenderator_index_task.json";
|
||||
private static final String REALTIME_QUERIES_RESOURCE = "/indexer/wikipedia_realtime_appenderator_index_queries.json";
|
||||
/**
|
||||
* The expected number of rows ingested for this test.
|
||||
* The total number of rows of raw data is 22 and there's no rollup.
|
||||
*/
|
||||
private static final int EXPECTED_NUM_ROWS = 22;
|
||||
|
||||
@Test
|
||||
public void testRealtimeIndexTask()
|
||||
{
|
||||
doTest();
|
||||
}
|
||||
|
||||
@Override
|
||||
void postEvents() throws Exception
|
||||
{
|
||||
final ServerDiscoverySelector eventReceiverSelector = factory.createSelector(EVENT_RECEIVER_SERVICE_NAME);
|
||||
eventReceiverSelector.start();
|
||||
InputStreamReader isr;
|
||||
try {
|
||||
isr = new InputStreamReader(
|
||||
ITRealtimeIndexTaskTest.class.getResourceAsStream(EVENT_DATA_FILE),
|
||||
StandardCharsets.UTF_8
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
try (BufferedReader reader = new BufferedReader(isr)) {
|
||||
ServerDiscoveryUtil.waitUntilInstanceReady(eventReceiverSelector, "Event Receiver");
|
||||
// Use the host from the config file and the port announced in zookeeper
|
||||
String host = config.getMiddleManagerHost() + ":" + eventReceiverSelector.pick().getPort();
|
||||
LOG.info("Event Receiver Found at host [%s]", host);
|
||||
EventReceiverFirehoseTestClient client = new EventReceiverFirehoseTestClient(
|
||||
host,
|
||||
EVENT_RECEIVER_SERVICE_NAME,
|
||||
jsonMapper,
|
||||
httpClient,
|
||||
smileMapper
|
||||
);
|
||||
// there are 22 lines in the file
|
||||
int i = 1;
|
||||
DateTime dt = DateTimes.nowUtc(); // timestamp used for sending each event
|
||||
dtFirst = dt;
|
||||
dtLast = dt;
|
||||
String line;
|
||||
while ((line = reader.readLine()) != null) {
|
||||
if (i == 15) { // for the 15th line, use a time before the window
|
||||
dt = dt.minusMinutes(10);
|
||||
dtFirst = dt; // oldest timestamp
|
||||
} else if (i == 16) { // remember this time to use in the expected response from the groupBy query
|
||||
dtGroupBy = dt;
|
||||
} else if (i == 18) { // use a time 6 seconds ago so it will be out of order
|
||||
dt = dt.minusSeconds(6);
|
||||
}
|
||||
String event = StringUtils.replace(line, TIME_PLACEHOLDER, EVENT_FMT.print(dt));
|
||||
LOG.info("sending event: [%s]\n", event);
|
||||
Collection<Map<String, Object>> events = new ArrayList<>();
|
||||
events.add(this.jsonMapper.readValue(event, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT));
|
||||
int eventsPosted = client.postEvents(events, this.jsonMapper, MediaType.APPLICATION_JSON);
|
||||
if (eventsPosted != events.size()) {
|
||||
throw new ISE("Event not posted");
|
||||
}
|
||||
|
||||
try {
|
||||
Thread.sleep(DELAY_BETWEEN_EVENTS_SECS * 1000);
|
||||
}
|
||||
catch (InterruptedException ignored) {
|
||||
/* nothing */
|
||||
}
|
||||
dtLast = dt; // latest timestamp
|
||||
dt = DateTimes.nowUtc();
|
||||
i++;
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
finally {
|
||||
eventReceiverSelector.stop();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
String getTaskResource()
|
||||
{
|
||||
return REALTIME_TASK_RESOURCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
String getQueriesResource()
|
||||
{
|
||||
return REALTIME_QUERIES_RESOURCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
int getNumExpectedRowsIngested()
|
||||
{
|
||||
return EXPECTED_NUM_ROWS;
|
||||
}
|
||||
}
|
|
@ -1,152 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.tests.indexer;
|
||||
|
||||
import org.apache.druid.curator.discovery.ServerDiscoverySelector;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
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.testing.clients.EventReceiverFirehoseTestClient;
|
||||
import org.apache.druid.testing.guice.DruidTestModuleFactory;
|
||||
import org.apache.druid.testing.utils.ServerDiscoveryUtil;
|
||||
import org.apache.druid.tests.TestNGGroup;
|
||||
import org.joda.time.DateTime;
|
||||
import org.testng.annotations.Guice;
|
||||
import org.testng.annotations.Test;
|
||||
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import java.io.BufferedReader;
|
||||
import java.io.InputStreamReader;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* See {@link AbstractITRealtimeIndexTaskTest} for test details.
|
||||
*/
|
||||
@Test(groups = {TestNGGroup.REALTIME_INDEX, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_METADATA_QUERY_DISABLED})
|
||||
@Guice(moduleFactory = DruidTestModuleFactory.class)
|
||||
public class ITRealtimeIndexTaskTest extends AbstractITRealtimeIndexTaskTest
|
||||
{
|
||||
private static final Logger LOG = new Logger(ITRealtimeIndexTaskTest.class);
|
||||
private static final String REALTIME_TASK_RESOURCE = "/indexer/wikipedia_realtime_index_task.json";
|
||||
private static final String REALTIME_QUERIES_RESOURCE = "/indexer/wikipedia_realtime_index_queries.json";
|
||||
/**
|
||||
* The expected number of rows ingested for this test.
|
||||
* The total number of rows of raw data is 22, but two rows will be rolled up into one row.
|
||||
*/
|
||||
private static final int EXPECTED_NUM_ROWS = 21;
|
||||
|
||||
@Test
|
||||
public void testRealtimeIndexTask()
|
||||
{
|
||||
doTest();
|
||||
}
|
||||
|
||||
@Override
|
||||
String getTaskResource()
|
||||
{
|
||||
return REALTIME_TASK_RESOURCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
String getQueriesResource()
|
||||
{
|
||||
return REALTIME_QUERIES_RESOURCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
int getNumExpectedRowsIngested()
|
||||
{
|
||||
return EXPECTED_NUM_ROWS;
|
||||
}
|
||||
|
||||
@Override
|
||||
void postEvents() throws Exception
|
||||
{
|
||||
final ServerDiscoverySelector eventReceiverSelector = factory.createSelector(EVENT_RECEIVER_SERVICE_NAME);
|
||||
eventReceiverSelector.start();
|
||||
InputStreamReader isr;
|
||||
try {
|
||||
isr = new InputStreamReader(
|
||||
ITRealtimeIndexTaskTest.class.getResourceAsStream(EVENT_DATA_FILE),
|
||||
StandardCharsets.UTF_8
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
try (BufferedReader reader = new BufferedReader(isr)) {
|
||||
ServerDiscoveryUtil.waitUntilInstanceReady(eventReceiverSelector, "Event Receiver");
|
||||
// Use the host from the config file and the port announced in zookeeper
|
||||
String host = config.getMiddleManagerHost() + ":" + eventReceiverSelector.pick().getPort();
|
||||
LOG.info("Event Receiver Found at host [%s]", host);
|
||||
EventReceiverFirehoseTestClient client = new EventReceiverFirehoseTestClient(
|
||||
host,
|
||||
EVENT_RECEIVER_SERVICE_NAME,
|
||||
jsonMapper,
|
||||
httpClient,
|
||||
smileMapper
|
||||
);
|
||||
// there are 22 lines in the file
|
||||
int i = 1;
|
||||
DateTime dt = DateTimes.nowUtc(); // timestamp used for sending each event
|
||||
dtFirst = dt; // timestamp of 1st event
|
||||
dtLast = dt; // timestamp of last event
|
||||
String line;
|
||||
while ((line = reader.readLine()) != null) {
|
||||
if (i == 15) { // for the 15th line, use a time before the window
|
||||
dt = dt.minusMinutes(10);
|
||||
} else if (i == 16) { // remember this time to use in the expected response from the groupBy query
|
||||
dtGroupBy = dt;
|
||||
} else if (i == 18) { // use a time 6 seconds ago so it will be out of order
|
||||
dt = dt.minusSeconds(6);
|
||||
}
|
||||
String event = StringUtils.replace(line, TIME_PLACEHOLDER, EVENT_FMT.print(dt));
|
||||
LOG.info("sending event: [%s]\n", event);
|
||||
Collection<Map<String, Object>> events = new ArrayList<>();
|
||||
events.add(this.jsonMapper.readValue(event, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT));
|
||||
int eventsPosted = client.postEvents(events, this.jsonMapper, MediaType.APPLICATION_JSON);
|
||||
if (eventsPosted != events.size()) {
|
||||
throw new ISE("Event not posted");
|
||||
}
|
||||
|
||||
try {
|
||||
Thread.sleep(DELAY_BETWEEN_EVENTS_SECS * 1000);
|
||||
}
|
||||
catch (InterruptedException ignored) {
|
||||
/* nothing */
|
||||
}
|
||||
dtLast = dt;
|
||||
dt = DateTimes.nowUtc();
|
||||
i++;
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
finally {
|
||||
eventReceiverSelector.stop();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,248 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.tests.query;
|
||||
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.druid.curator.discovery.ServerDiscoveryFactory;
|
||||
import org.apache.druid.curator.discovery.ServerDiscoverySelector;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
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.query.aggregation.LongSumAggregatorFactory;
|
||||
import org.apache.druid.testing.IntegrationTestingConfig;
|
||||
import org.apache.druid.testing.clients.EventReceiverFirehoseTestClient;
|
||||
import org.apache.druid.testing.guice.DruidTestModuleFactory;
|
||||
import org.apache.druid.testing.guice.TestClient;
|
||||
import org.apache.druid.testing.utils.ITRetryUtil;
|
||||
import org.apache.druid.testing.utils.ServerDiscoveryUtil;
|
||||
import org.apache.druid.tests.TestNGGroup;
|
||||
import org.apache.druid.tests.indexer.AbstractITBatchIndexTest;
|
||||
import org.apache.druid.tests.indexer.AbstractIndexerTest;
|
||||
import org.jboss.netty.handler.codec.http.HttpMethod;
|
||||
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
|
||||
import org.joda.time.DateTime;
|
||||
import org.testng.annotations.BeforeSuite;
|
||||
import org.testng.annotations.Guice;
|
||||
import org.testng.annotations.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.URL;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
@Test(groups = {TestNGGroup.QUERY, TestNGGroup.CENTRALIZED_DATASOURCE_SCHEMA})
|
||||
@Guice(moduleFactory = DruidTestModuleFactory.class)
|
||||
public class ITUnionQueryTest extends AbstractIndexerTest
|
||||
{
|
||||
private static final Logger LOG = new Logger(ITUnionQueryTest.class);
|
||||
private static final String UNION_TASK_RESOURCE = "/indexer/wikipedia_union_index_task.json";
|
||||
private static final String EVENT_RECEIVER_SERVICE_PREFIX = "eventReceiverServiceName";
|
||||
private static final String UNION_DATA_FILE = "/data/union_query/wikipedia_index_data.json";
|
||||
private static final String UNION_QUERIES_RESOURCE = "/queries/union_queries.json";
|
||||
private static final String UNION_DATASOURCE = "wikipedia_index_test";
|
||||
|
||||
@Inject
|
||||
ServerDiscoveryFactory factory;
|
||||
|
||||
@Inject
|
||||
@TestClient
|
||||
HttpClient httpClient;
|
||||
|
||||
@Inject
|
||||
IntegrationTestingConfig config;
|
||||
|
||||
private String fullDatasourceName;
|
||||
|
||||
@BeforeSuite
|
||||
public void setFullDatasourceName()
|
||||
{
|
||||
fullDatasourceName = UNION_DATASOURCE + config.getExtraDatasourceNameSuffix();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnionQuery() throws IOException
|
||||
{
|
||||
final int numTasks = 3;
|
||||
final Closer closer = Closer.create();
|
||||
for (int i = 0; i < numTasks; i++) {
|
||||
closer.register(unloader(fullDatasourceName + i));
|
||||
}
|
||||
try {
|
||||
// Load 3 datasources with same dimensions
|
||||
String task = setShutOffTime(
|
||||
getResourceAsString(UNION_TASK_RESOURCE),
|
||||
DateTimes.utc(System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(3))
|
||||
);
|
||||
List<String> taskIDs = new ArrayList<>();
|
||||
for (int i = 0; i < numTasks; i++) {
|
||||
taskIDs.add(
|
||||
indexer.submitTask(
|
||||
withServiceName(
|
||||
withDataSource(task, fullDatasourceName + i),
|
||||
EVENT_RECEIVER_SERVICE_PREFIX + i
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
for (int i = 0; i < numTasks; i++) {
|
||||
postEvents(i);
|
||||
}
|
||||
|
||||
// wait until all events are ingested
|
||||
ITRetryUtil.retryUntil(
|
||||
() -> {
|
||||
for (int i = 0; i < numTasks; i++) {
|
||||
final int countRows = queryHelper.countRows(
|
||||
fullDatasourceName + i,
|
||||
Intervals.of("2013-08-31/2013-09-01"),
|
||||
name -> new LongSumAggregatorFactory(name, "count")
|
||||
);
|
||||
|
||||
// there are 10 rows, but query only covers the first 5
|
||||
if (countRows < 5) {
|
||||
LOG.warn("%d events have been ingested to %s so far", countRows, fullDatasourceName + i);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
},
|
||||
true,
|
||||
1000,
|
||||
100,
|
||||
"Waiting all events are ingested"
|
||||
);
|
||||
|
||||
// should hit the queries on realtime task
|
||||
LOG.info("Running Union Queries..");
|
||||
|
||||
String queryResponseTemplate;
|
||||
try {
|
||||
InputStream is = AbstractITBatchIndexTest.class.getResourceAsStream(UNION_QUERIES_RESOURCE);
|
||||
queryResponseTemplate = IOUtils.toString(is, StandardCharsets.UTF_8);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new ISE(e, "could not read query file: %s", UNION_QUERIES_RESOURCE);
|
||||
}
|
||||
|
||||
queryResponseTemplate = StringUtils.replace(
|
||||
queryResponseTemplate,
|
||||
"%%DATASOURCE%%",
|
||||
fullDatasourceName
|
||||
);
|
||||
|
||||
this.queryHelper.testQueriesFromString(queryResponseTemplate);
|
||||
|
||||
// wait for the task to complete
|
||||
for (int i = 0; i < numTasks; i++) {
|
||||
indexer.waitUntilTaskCompletes(taskIDs.get(i));
|
||||
}
|
||||
// task should complete only after the segments are loaded by historical node
|
||||
for (int i = 0; i < numTasks; i++) {
|
||||
final int taskNum = i;
|
||||
ITRetryUtil.retryUntil(
|
||||
() -> coordinator.areSegmentsLoaded(fullDatasourceName + taskNum),
|
||||
true,
|
||||
10000,
|
||||
10,
|
||||
"Real-time generated segments loaded"
|
||||
);
|
||||
}
|
||||
// run queries on historical nodes
|
||||
this.queryHelper.testQueriesFromString(queryResponseTemplate);
|
||||
|
||||
}
|
||||
catch (Throwable e) {
|
||||
throw closer.rethrow(e);
|
||||
}
|
||||
finally {
|
||||
closer.close();
|
||||
}
|
||||
}
|
||||
|
||||
private String setShutOffTime(String taskAsString, DateTime time)
|
||||
{
|
||||
return StringUtils.replace(taskAsString, "#SHUTOFFTIME", time.toString());
|
||||
}
|
||||
|
||||
private String withDataSource(String taskAsString, String dataSource)
|
||||
{
|
||||
return StringUtils.replace(taskAsString, "%%DATASOURCE%%", dataSource);
|
||||
}
|
||||
|
||||
private String withServiceName(String taskAsString, String serviceName)
|
||||
{
|
||||
return StringUtils.replace(taskAsString, EVENT_RECEIVER_SERVICE_PREFIX, serviceName);
|
||||
}
|
||||
|
||||
private void postEvents(int id) throws Exception
|
||||
{
|
||||
final ServerDiscoverySelector eventReceiverSelector = factory.createSelector(EVENT_RECEIVER_SERVICE_PREFIX + id);
|
||||
eventReceiverSelector.start();
|
||||
try {
|
||||
ServerDiscoveryUtil.waitUntilInstanceReady(eventReceiverSelector, "Event Receiver");
|
||||
// Access the docker VM mapped host and port instead of service announced in zookeeper
|
||||
String host = config.getMiddleManagerHost() + ":" + eventReceiverSelector.pick().getPort();
|
||||
|
||||
LOG.info("Event Receiver Found at host [%s]", host);
|
||||
|
||||
LOG.info("Checking worker /status/health for [%s]", host);
|
||||
ITRetryUtil.retryUntilTrue(
|
||||
() -> {
|
||||
try {
|
||||
StatusResponseHolder response = httpClient.go(
|
||||
new Request(HttpMethod.GET, new URL(StringUtils.format("https://%s/status/health", host))),
|
||||
StatusResponseHandler.getInstance()
|
||||
).get();
|
||||
return response.getStatus().equals(HttpResponseStatus.OK);
|
||||
}
|
||||
catch (Throwable e) {
|
||||
LOG.error(e, "");
|
||||
return false;
|
||||
}
|
||||
},
|
||||
StringUtils.format("Checking /status/health for worker [%s]", host)
|
||||
);
|
||||
LOG.info("Finished checking worker /status/health for [%s], success", host);
|
||||
|
||||
EventReceiverFirehoseTestClient client = new EventReceiverFirehoseTestClient(
|
||||
host,
|
||||
EVENT_RECEIVER_SERVICE_PREFIX + id,
|
||||
jsonMapper,
|
||||
httpClient,
|
||||
smileMapper
|
||||
);
|
||||
client.postEventsFromFile(UNION_DATA_FILE);
|
||||
}
|
||||
finally {
|
||||
eventReceiverSelector.stop();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,10 @@
|
|||
{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143}
|
||||
{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330}
|
||||
{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111}
|
||||
{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
|
||||
{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9}
|
||||
{"timestamp": "2013-09-01T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143}
|
||||
{"timestamp": "2013-09-01T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330}
|
||||
{"timestamp": "2013-09-01T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111}
|
||||
{"timestamp": "2013-09-01T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
|
||||
{"timestamp": "2013-09-01T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9}
|
|
@ -61,7 +61,7 @@
|
|||
"type": "index_parallel",
|
||||
"inputSource": {
|
||||
"type": "local",
|
||||
"baseDir": "/resources/data/union_query/",
|
||||
"baseDir": "/resources/data/broadcast/",
|
||||
"filter": "wikipedia_index_data*"
|
||||
},
|
||||
"appendToExisting": false,
|
||||
|
|
|
@ -1,94 +0,0 @@
|
|||
{
|
||||
"type": "index_realtime_appenderator",
|
||||
"spec": {
|
||||
"dataSchema": {
|
||||
"dataSource": "%%DATASOURCE%%",
|
||||
"metricsSpec": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "count"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "added",
|
||||
"fieldName": "added"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "deleted",
|
||||
"fieldName": "deleted"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "delta",
|
||||
"fieldName": "delta"
|
||||
}
|
||||
],
|
||||
"granularitySpec": {
|
||||
"segmentGranularity": "minute",
|
||||
"queryGranularity": "second"
|
||||
},
|
||||
"parser": {
|
||||
"type": "map",
|
||||
"parseSpec": {
|
||||
"format": "tsv",
|
||||
"columns": [
|
||||
"timestamp",
|
||||
"page",
|
||||
"language",
|
||||
"user",
|
||||
"unpatrolled",
|
||||
"newPage",
|
||||
"robot",
|
||||
"anonymous",
|
||||
"namespace",
|
||||
"continent",
|
||||
"country",
|
||||
"region",
|
||||
"city",
|
||||
"added",
|
||||
"deleted",
|
||||
"delta"
|
||||
],
|
||||
"timestampSpec": {
|
||||
"column": "timestamp",
|
||||
"format": "iso"
|
||||
},
|
||||
"dimensionsSpec": {
|
||||
"dimensions": [
|
||||
"page",
|
||||
"language",
|
||||
"user",
|
||||
"unpatrolled",
|
||||
"newPage",
|
||||
"robot",
|
||||
"anonymous",
|
||||
"namespace",
|
||||
"continent",
|
||||
"country",
|
||||
"region",
|
||||
"city"
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"ioConfig": {
|
||||
"type": "realtime",
|
||||
"firehose": {
|
||||
"type": "timed",
|
||||
"shutoffTime": "#SHUTOFFTIME",
|
||||
"delegate": {
|
||||
"type": "receiver",
|
||||
"serviceName": "eventReceiverServiceName",
|
||||
"bufferSize": 100000
|
||||
}
|
||||
}
|
||||
},
|
||||
"tuningConfig": {
|
||||
"type": "realtime_appenderator",
|
||||
"maxRowsInMemory": 1,
|
||||
"intermediatePersistPeriod": "PT1M"
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,22 +0,0 @@
|
|||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 1, "deleted": 1, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 3, "deleted": 3, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 5, "deleted": 5, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 7, "deleted": 3, "delta": 4}
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 11, "deleted": 11, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 13, "deleted": 13, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 17, "deleted": 17, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 19, "deleted": 19, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 23, "deleted": 23, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 29, "deleted": 31, "delta": -1}
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 31, "deleted": 31, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 37, "deleted": 37, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 41, "deleted": 41, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 43, "deleted": 43, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 47, "deleted": 47, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 59, "deleted": 59, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 61, "deleted": 61, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 67, "deleted": 67, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 71, "deleted": 71, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 73, "deleted": 73, "delta": 0},
|
||||
{"timestamp": "YYYY-MM-DDTHH:MM:SSZ", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 79, "deleted": 79, "delta": 0}
|
|
@ -1,98 +0,0 @@
|
|||
{
|
||||
"type": "index_realtime",
|
||||
"spec": {
|
||||
"dataSchema": {
|
||||
"dataSource": "%%DATASOURCE%%",
|
||||
"metricsSpec": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "count"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "added",
|
||||
"fieldName": "added"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "deleted",
|
||||
"fieldName": "deleted"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "delta",
|
||||
"fieldName": "delta"
|
||||
}
|
||||
],
|
||||
"granularitySpec": {
|
||||
"segmentGranularity": "minute",
|
||||
"queryGranularity": "second"
|
||||
},
|
||||
"parser": {
|
||||
"type": "map",
|
||||
"parseSpec": {
|
||||
"format": "tsv",
|
||||
"columns": [
|
||||
"timestamp",
|
||||
"page",
|
||||
"language",
|
||||
"user",
|
||||
"unpatrolled",
|
||||
"newPage",
|
||||
"robot",
|
||||
"anonymous",
|
||||
"namespace",
|
||||
"continent",
|
||||
"country",
|
||||
"region",
|
||||
"city",
|
||||
"added",
|
||||
"deleted",
|
||||
"delta"
|
||||
],
|
||||
"timestampSpec": {
|
||||
"column": "timestamp",
|
||||
"format": "iso"
|
||||
},
|
||||
"dimensionsSpec": {
|
||||
"dimensions": [
|
||||
"page",
|
||||
"language",
|
||||
"user",
|
||||
"unpatrolled",
|
||||
"newPage",
|
||||
"robot",
|
||||
"anonymous",
|
||||
"namespace",
|
||||
"continent",
|
||||
"country",
|
||||
"region",
|
||||
"city"
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"ioConfig": {
|
||||
"type": "realtime",
|
||||
"firehose": {
|
||||
"type": "timed",
|
||||
"shutoffTime": "#SHUTOFFTIME",
|
||||
"delegate": {
|
||||
"type": "receiver",
|
||||
"serviceName": "eventReceiverServiceName",
|
||||
"bufferSize": 100000
|
||||
}
|
||||
}
|
||||
},
|
||||
"tuningConfig": {
|
||||
"type": "realtime",
|
||||
"maxRowsInMemory": 1,
|
||||
"intermediatePersistPeriod": "PT1M",
|
||||
"windowPeriod": "PT1M",
|
||||
"rejectionPolicy": {
|
||||
"type": "serverTime"
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,98 +0,0 @@
|
|||
{
|
||||
"type": "index_realtime",
|
||||
"spec": {
|
||||
"dataSchema": {
|
||||
"dataSource": "%%DATASOURCE%%",
|
||||
"metricsSpec": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "count"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "added",
|
||||
"fieldName": "added"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "deleted",
|
||||
"fieldName": "deleted"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "delta",
|
||||
"fieldName": "delta"
|
||||
}
|
||||
],
|
||||
"granularitySpec": {
|
||||
"segmentGranularity": "DAY",
|
||||
"queryGranularity": "second"
|
||||
},
|
||||
"parser": {
|
||||
"type": "map",
|
||||
"parseSpec": {
|
||||
"format": "tsv",
|
||||
"columns": [
|
||||
"timestamp",
|
||||
"page",
|
||||
"language",
|
||||
"user",
|
||||
"unpatrolled",
|
||||
"newPage",
|
||||
"robot",
|
||||
"anonymous",
|
||||
"namespace",
|
||||
"continent",
|
||||
"country",
|
||||
"region",
|
||||
"city",
|
||||
"added",
|
||||
"deleted",
|
||||
"delta"
|
||||
],
|
||||
"timestampSpec": {
|
||||
"column": "timestamp",
|
||||
"format": "iso"
|
||||
},
|
||||
"dimensionsSpec": {
|
||||
"dimensions": [
|
||||
"page",
|
||||
"language",
|
||||
"user",
|
||||
"unpatrolled",
|
||||
"newPage",
|
||||
"robot",
|
||||
"anonymous",
|
||||
"namespace",
|
||||
"continent",
|
||||
"country",
|
||||
"region",
|
||||
"city"
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"ioConfig": {
|
||||
"type": "realtime",
|
||||
"firehose": {
|
||||
"type": "timed",
|
||||
"shutoffTime": "#SHUTOFFTIME",
|
||||
"delegate": {
|
||||
"type": "receiver",
|
||||
"serviceName": "eventReceiverServiceName",
|
||||
"bufferSize": 100000
|
||||
}
|
||||
}
|
||||
},
|
||||
"tuningConfig": {
|
||||
"type": "realtime",
|
||||
"maxRowsInMemory": 1,
|
||||
"intermediatePersistPeriod": "PT1M",
|
||||
"windowPeriod": "PT1M",
|
||||
"rejectionPolicy": {
|
||||
"type": "none"
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,566 +0,0 @@
|
|||
[
|
||||
{
|
||||
"description": "timeseries, filtered, all aggs, all",
|
||||
"query": {
|
||||
"queryType": "timeseries",
|
||||
"dataSource": {
|
||||
"type": "union",
|
||||
"dataSources": [
|
||||
"%%DATASOURCE%%1", "%%DATASOURCE%%2", "%%DATASOURCE%%3",
|
||||
"%%DATASOURCE%%0"
|
||||
]
|
||||
},
|
||||
"intervals": ["2013-08-31/2013-09-01"],
|
||||
"granularity": "all",
|
||||
"filter": {
|
||||
"type": "selector",
|
||||
"dimension": "language",
|
||||
"value": "en"
|
||||
},
|
||||
"aggregations": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "rows"
|
||||
},
|
||||
{
|
||||
"type": "longSum",
|
||||
"fieldName": "count",
|
||||
"name": "count"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "added",
|
||||
"name": "added"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "deleted",
|
||||
"name": "deleted"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "delta",
|
||||
"name": "delta"
|
||||
}
|
||||
],
|
||||
"context": {
|
||||
"useCache": "true",
|
||||
"populateCache": "true",
|
||||
"timeout": 60000
|
||||
}
|
||||
},
|
||||
"expectedResults": [
|
||||
{
|
||||
"timestamp": "2013-08-31T01:02:33.000Z",
|
||||
"result": {
|
||||
"added": 1548.0,
|
||||
"count": 6,
|
||||
"delta": 561.0,
|
||||
"deleted": 987.0,
|
||||
"rows": 6
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"description": "topN, all aggs, page dim, uniques metric",
|
||||
"query": {
|
||||
"queryType": "topN",
|
||||
"dataSource": {
|
||||
"type": "union",
|
||||
"dataSources": [
|
||||
"%%DATASOURCE%%1", "%%DATASOURCE%%2", "%%DATASOURCE%%3",
|
||||
"%%DATASOURCE%%0"
|
||||
]
|
||||
},
|
||||
"intervals": ["2013-08-31/2013-09-01"],
|
||||
"granularity": "all",
|
||||
"aggregations": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "rows"
|
||||
},
|
||||
{
|
||||
"type": "longSum",
|
||||
"fieldName": "count",
|
||||
"name": "count"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "added",
|
||||
"name": "added"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "deleted",
|
||||
"name": "deleted"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "delta",
|
||||
"name": "delta"
|
||||
}
|
||||
],
|
||||
"dimension": "page",
|
||||
"metric": "added",
|
||||
"threshold": 3,
|
||||
"context": {
|
||||
"useCache": "true",
|
||||
"populateCache": "true",
|
||||
"timeout": 60000
|
||||
}
|
||||
},
|
||||
"expectedResults": [
|
||||
{
|
||||
"timestamp": "2013-08-31T01:02:33.000Z",
|
||||
"result": [
|
||||
{
|
||||
"added": 2715.0,
|
||||
"count": 3,
|
||||
"page": "Crimson Typhoon",
|
||||
"delta": 2700.0,
|
||||
"deleted": 15.0,
|
||||
"rows": 3
|
||||
},
|
||||
{
|
||||
"added": 1377.0,
|
||||
"count": 3,
|
||||
"page": "Striker Eureka",
|
||||
"delta": 990.0,
|
||||
"deleted": 387.0,
|
||||
"rows": 3
|
||||
},
|
||||
{
|
||||
"added": 369.0,
|
||||
"count": 3,
|
||||
"page": "Cherno Alpha",
|
||||
"delta": 333.0,
|
||||
"deleted": 36.0,
|
||||
"rows": 3
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"description": "topN, all aggs, page dim, count metric, postAggs",
|
||||
"query": {
|
||||
"queryType": "topN",
|
||||
"dataSource": {
|
||||
"type": "union",
|
||||
"dataSources": [
|
||||
"%%DATASOURCE%%1", "%%DATASOURCE%%2", "%%DATASOURCE%%3",
|
||||
"%%DATASOURCE%%0"
|
||||
]
|
||||
},
|
||||
"intervals": ["2013-08-31/2013-09-01"],
|
||||
"granularity": "all",
|
||||
"aggregations": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "rows"
|
||||
},
|
||||
{
|
||||
"type": "longSum",
|
||||
"fieldName": "count",
|
||||
"name": "count"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "added",
|
||||
"name": "added"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "deleted",
|
||||
"name": "deleted"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "delta",
|
||||
"name": "delta"
|
||||
}
|
||||
],
|
||||
"postAggregations": [
|
||||
{
|
||||
"type": "arithmetic",
|
||||
"name": "sumOfAddedDeletedConst",
|
||||
"fn": "+",
|
||||
"fields": [
|
||||
{
|
||||
"type": "fieldAccess",
|
||||
"name": "added",
|
||||
"fieldName": "added"
|
||||
},
|
||||
{
|
||||
"type": "arithmetic",
|
||||
"name": "",
|
||||
"fn": "+",
|
||||
"fields": [
|
||||
{
|
||||
"type": "fieldAccess",
|
||||
"name": "deleted",
|
||||
"fieldName": "deleted"
|
||||
},
|
||||
{
|
||||
"type": "constant",
|
||||
"name": "constant",
|
||||
"value": 1000
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
],
|
||||
"dimension": "page",
|
||||
"metric": "added",
|
||||
"threshold": 3,
|
||||
"context": {
|
||||
"useCache": "true",
|
||||
"populateCache": "true",
|
||||
"timeout": 60000
|
||||
}
|
||||
},
|
||||
"expectedResults": [
|
||||
{
|
||||
"timestamp": "2013-08-31T01:02:33.000Z",
|
||||
"result": [
|
||||
{
|
||||
"added": 2715.0,
|
||||
"count": 3,
|
||||
"page": "Crimson Typhoon",
|
||||
"delta": 2700.0,
|
||||
"deleted": 15.0,
|
||||
"sumOfAddedDeletedConst": 3730.0,
|
||||
"rows": 3
|
||||
},
|
||||
{
|
||||
"added": 1377.0,
|
||||
"count": 3,
|
||||
"page": "Striker Eureka",
|
||||
"delta": 990.0,
|
||||
"deleted": 387.0,
|
||||
"sumOfAddedDeletedConst": 2764.0,
|
||||
"rows": 3
|
||||
},
|
||||
{
|
||||
"added": 369.0,
|
||||
"count": 3,
|
||||
"page": "Cherno Alpha",
|
||||
"delta": 333.0,
|
||||
"deleted": 36.0,
|
||||
"sumOfAddedDeletedConst": 1405.0,
|
||||
"rows": 3
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"description": "topN, lexicographic, two aggs, language dim, postAggs",
|
||||
"query": {
|
||||
"queryType": "topN",
|
||||
"dataSource": {
|
||||
"type": "union",
|
||||
"dataSources": [
|
||||
"%%DATASOURCE%%1", "%%DATASOURCE%%2", "%%DATASOURCE%%3",
|
||||
"%%DATASOURCE%%0"
|
||||
]
|
||||
},
|
||||
"intervals": ["2013-08-31/2013-09-01"],
|
||||
"granularity": "all",
|
||||
"aggregations": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "rows"
|
||||
},
|
||||
{
|
||||
"type": "longSum",
|
||||
"fieldName": "count",
|
||||
"name": "count"
|
||||
}
|
||||
],
|
||||
"postAggregations": [
|
||||
{
|
||||
"type": "arithmetic",
|
||||
"name": "sumOfRowsAndCount",
|
||||
"fn": "+",
|
||||
"fields": [
|
||||
{
|
||||
"type": "fieldAccess",
|
||||
"name": "rows",
|
||||
"fieldName": "rows"
|
||||
},
|
||||
{
|
||||
"type": "fieldAccess",
|
||||
"name": "count",
|
||||
"fieldName": "count"
|
||||
}
|
||||
]
|
||||
}
|
||||
],
|
||||
"dimension": "language",
|
||||
"metric": {
|
||||
"type": "lexicographic",
|
||||
"previousStop": "a"
|
||||
},
|
||||
"threshold": 3,
|
||||
"context": {
|
||||
"useCache": "true",
|
||||
"populateCache": "true",
|
||||
"timeout": 60000
|
||||
}
|
||||
},
|
||||
"expectedResults": [
|
||||
{
|
||||
"timestamp": "2013-08-31T01:02:33.000Z",
|
||||
"result": [
|
||||
{
|
||||
"sumOfRowsAndCount": 12.0,
|
||||
"count": 6,
|
||||
"language": "en",
|
||||
"rows": 6
|
||||
},
|
||||
{
|
||||
"sumOfRowsAndCount": 6.0,
|
||||
"count": 3,
|
||||
"language": "ja",
|
||||
"rows": 3
|
||||
},
|
||||
{
|
||||
"sumOfRowsAndCount": 6.0,
|
||||
"count": 3,
|
||||
"language": "ru",
|
||||
"rows": 3
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"description": "groupBy, two aggs, namespace dim, postAggs",
|
||||
"query": {
|
||||
"queryType": "groupBy",
|
||||
"dataSource": {
|
||||
"type": "union",
|
||||
"dataSources": [
|
||||
"%%DATASOURCE%%1", "%%DATASOURCE%%2", "%%DATASOURCE%%3",
|
||||
"%%DATASOURCE%%0"
|
||||
]
|
||||
},
|
||||
"intervals": ["2013-08-31/2013-09-01"],
|
||||
"granularity": "all",
|
||||
"aggregations": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "rows"
|
||||
},
|
||||
{
|
||||
"type": "longSum",
|
||||
"fieldName": "count",
|
||||
"name": "count"
|
||||
}
|
||||
],
|
||||
"postAggregations": [
|
||||
{
|
||||
"type": "arithmetic",
|
||||
"name": "sumOfRowsAndCount",
|
||||
"fn": "+",
|
||||
"fields": [
|
||||
{
|
||||
"type": "fieldAccess",
|
||||
"name": "rows",
|
||||
"fieldName": "rows"
|
||||
},
|
||||
{
|
||||
"type": "fieldAccess",
|
||||
"name": "count",
|
||||
"fieldName": "count"
|
||||
}
|
||||
]
|
||||
}
|
||||
],
|
||||
"dimensions": ["namespace"],
|
||||
"context": {
|
||||
"useCache": "true",
|
||||
"populateCache": "true",
|
||||
"timeout": 60000
|
||||
}
|
||||
},
|
||||
"expectedResults": [
|
||||
{
|
||||
"version": "v1",
|
||||
"timestamp": "2013-08-31T00:00:00.000Z",
|
||||
"event": {
|
||||
"sumOfRowsAndCount": 12.0,
|
||||
"count": 6,
|
||||
"rows": 6,
|
||||
"namespace": "article"
|
||||
}
|
||||
},
|
||||
{
|
||||
"version": "v1",
|
||||
"timestamp": "2013-08-31T00:00:00.000Z",
|
||||
"event": {
|
||||
"sumOfRowsAndCount": 18.0,
|
||||
"count": 9,
|
||||
"rows": 9,
|
||||
"namespace": "wikipedia"
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"description": "groupBy, two aggs, namespace + robot dim, postAggs",
|
||||
"query": {
|
||||
"queryType": "groupBy",
|
||||
"dataSource": {
|
||||
"type": "union",
|
||||
"dataSources": [
|
||||
"%%DATASOURCE%%1", "%%DATASOURCE%%2", "%%DATASOURCE%%3",
|
||||
"%%DATASOURCE%%0"
|
||||
]
|
||||
},
|
||||
"intervals": ["2013-08-31/2013-09-01"],
|
||||
"granularity": "all",
|
||||
"aggregations": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "rows"
|
||||
},
|
||||
{
|
||||
"type": "longSum",
|
||||
"fieldName": "count",
|
||||
"name": "count"
|
||||
}
|
||||
],
|
||||
"postAggregations": [
|
||||
{
|
||||
"type": "arithmetic",
|
||||
"name": "sumOfRowsAndCount",
|
||||
"fn": "+",
|
||||
"fields": [
|
||||
{
|
||||
"type": "fieldAccess",
|
||||
"name": "rows",
|
||||
"fieldName": "rows"
|
||||
},
|
||||
{
|
||||
"type": "fieldAccess",
|
||||
"name": "count",
|
||||
"fieldName": "count"
|
||||
}
|
||||
]
|
||||
}
|
||||
],
|
||||
"dimensions": ["namespace", "robot"],
|
||||
"limitSpec": {
|
||||
"type": "default",
|
||||
"limit": 3,
|
||||
"orderBy": ["robot", "namespace"]
|
||||
},
|
||||
"context": {
|
||||
"useCache": "true",
|
||||
"populateCache": "true",
|
||||
"timeout": 60000
|
||||
}
|
||||
},
|
||||
"expectedResults": [
|
||||
{
|
||||
"version": "v1",
|
||||
"timestamp": "2013-08-31T00:00:00.000Z",
|
||||
"event": {
|
||||
"sumOfRowsAndCount": 6.0,
|
||||
"count": 3,
|
||||
"robot": "false",
|
||||
"rows": 3,
|
||||
"namespace": "article"
|
||||
}
|
||||
},
|
||||
{
|
||||
"version": "v1",
|
||||
"timestamp": "2013-08-31T00:00:00.000Z",
|
||||
"event": {
|
||||
"sumOfRowsAndCount": 6.0,
|
||||
"count": 3,
|
||||
"robot": "true",
|
||||
"rows": 3,
|
||||
"namespace": "article"
|
||||
}
|
||||
},
|
||||
{
|
||||
"version": "v1",
|
||||
"timestamp": "2013-08-31T00:00:00.000Z",
|
||||
"event": {
|
||||
"sumOfRowsAndCount": 18.0,
|
||||
"count": 9,
|
||||
"robot": "true",
|
||||
"rows": 9,
|
||||
"namespace": "wikipedia"
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"query": {
|
||||
"queryType": "search",
|
||||
"intervals": ["2013-08-31/2013-09-01"],
|
||||
"dataSource": {
|
||||
"type": "union",
|
||||
"dataSources": [
|
||||
"%%DATASOURCE%%1", "%%DATASOURCE%%2", "%%DATASOURCE%%3",
|
||||
"%%DATASOURCE%%0"
|
||||
]
|
||||
},
|
||||
"granularity": "all",
|
||||
"query": {
|
||||
"type": "insensitive_contains",
|
||||
"value": "ip"
|
||||
},
|
||||
"context": {
|
||||
"useCache": "true",
|
||||
"populateCache": "true",
|
||||
"timeout": 60000
|
||||
}
|
||||
},
|
||||
"expectedResults": [
|
||||
{
|
||||
"timestamp": "2013-08-31T00:00:00.000Z",
|
||||
"result": [
|
||||
{
|
||||
"dimension": "user",
|
||||
"value": "triplets",
|
||||
"count":3
|
||||
},
|
||||
{
|
||||
"dimension": "namespace",
|
||||
"value": "wikipedia",
|
||||
"count":9
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"description": "timeboundary, 1 agg, union",
|
||||
"query": {
|
||||
"queryType": "timeBoundary",
|
||||
"dataSource": {
|
||||
"type": "union",
|
||||
"dataSources": [
|
||||
"%%DATASOURCE%%1", "%%DATASOURCE%%2", "%%DATASOURCE%%3",
|
||||
"%%DATASOURCE%%0"
|
||||
]
|
||||
}
|
||||
},
|
||||
"expectedResults": [
|
||||
{
|
||||
"timestamp": "2013-08-31T01:02:33.000Z",
|
||||
"result": {
|
||||
"minTime": "2013-08-31T01:02:33.000Z",
|
||||
"maxTime": "2013-09-01T12:41:27.000Z"
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
|
@ -1,50 +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.common.guava;
|
||||
|
||||
import java.util.concurrent.Callable;
|
||||
|
||||
public abstract class ThreadRenamingCallable<T> implements Callable<T>
|
||||
{
|
||||
private final String name;
|
||||
|
||||
public ThreadRenamingCallable(
|
||||
String name
|
||||
)
|
||||
{
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final T call()
|
||||
{
|
||||
final Thread currThread = Thread.currentThread();
|
||||
String currName = currThread.getName();
|
||||
try {
|
||||
currThread.setName(name);
|
||||
return doCall();
|
||||
}
|
||||
finally {
|
||||
currThread.setName(currName);
|
||||
}
|
||||
}
|
||||
|
||||
public abstract T doCall();
|
||||
}
|
|
@ -64,16 +64,6 @@ public class ScheduledExecutors
|
|||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Run callable repeatedly with the given delay between calls, after the given
|
||||
* initial delay, until it returns Signal.STOP. Exceptions are caught and
|
||||
* logged as errors.
|
||||
*/
|
||||
public static void scheduleWithFixedDelay(ScheduledExecutorService exec, Duration delay, Callable<Signal> callable)
|
||||
{
|
||||
scheduleWithFixedDelay(exec, delay, delay, callable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Run callable repeatedly with the given delay between calls, until it
|
||||
* returns Signal.STOP. Exceptions are caught and logged as errors.
|
||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.druid.segment.filter.DimensionPredicateFilter;
|
|||
import org.apache.druid.segment.filter.SelectorFilter;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
|
||||
|
|
|
@ -55,7 +55,6 @@ import org.joda.time.DateTime;
|
|||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
|
|
|
@ -39,7 +39,6 @@ import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
|||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
|
|
@ -108,11 +108,6 @@ public class Metadata
|
|||
return this;
|
||||
}
|
||||
|
||||
public Object get(String key)
|
||||
{
|
||||
return container.get(key);
|
||||
}
|
||||
|
||||
// arbitrary key-value pairs from the metadata just follow the semantics of last one wins if same
|
||||
// key exists in multiple input Metadata containers
|
||||
// for others e.g. Aggregators, appropriate merging is done
|
||||
|
|
|
@ -97,10 +97,4 @@ public class IncrementalIndexAddResult
|
|||
{
|
||||
return reasonOfNotAdded == null && parseException == null;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public String getReasonOfNotAdded()
|
||||
{
|
||||
return reasonOfNotAdded;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,8 +26,8 @@ import java.util.Map;
|
|||
|
||||
/**
|
||||
* A collection of meters for row ingestion stats, with support for moving average calculations.
|
||||
* This can eventually replace FireDepartmentMetrics, but moving averages for other stats collected by
|
||||
* FireDepartmentMetrics are not currently supported, so we continue to use FireDepartmentMetrics alongside
|
||||
* This can eventually replace SegmentGenerationMetrics, but moving averages for other stats collected by
|
||||
* SegmentGenerationMetrics are not currently supported, so we continue to use SegmentGenerationMetrics alongside
|
||||
* RowIngestionMeters to avoid unnecessary overhead from maintaining these moving averages.
|
||||
*/
|
||||
@ExtensionPoint
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.junit.Assert;
|
|||
import org.junit.Test;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.Function;
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.druid.query.operator.OffsetLimit;
|
|||
import org.apache.druid.query.rowsandcols.LazilyDecoratedRowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumnsTestBase;
|
||||
|
||||
import java.util.function.Function;
|
||||
|
||||
public class ColumnBasedFrameRowsAndColumnsTest extends RowsAndColumnsTestBase
|
||||
|
|
|
@ -1,35 +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 org.apache.druid.metadata.SegmentPublisherProvider;
|
||||
import org.apache.druid.segment.realtime.NoopSegmentPublisher;
|
||||
import org.apache.druid.segment.realtime.SegmentPublisher;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class NoopSegmentPublisherProvider implements SegmentPublisherProvider
|
||||
{
|
||||
@Override
|
||||
public SegmentPublisher get()
|
||||
{
|
||||
return new NoopSegmentPublisher();
|
||||
}
|
||||
}
|
|
@ -29,8 +29,6 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
|
|||
import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator;
|
||||
import org.apache.druid.metadata.MetadataRuleManager;
|
||||
import org.apache.druid.metadata.MetadataRuleManagerProvider;
|
||||
import org.apache.druid.metadata.MetadataSegmentPublisher;
|
||||
import org.apache.druid.metadata.MetadataSegmentPublisherProvider;
|
||||
import org.apache.druid.metadata.MetadataStorageActionHandlerFactory;
|
||||
import org.apache.druid.metadata.MetadataStorageConnector;
|
||||
import org.apache.druid.metadata.MetadataStorageProvider;
|
||||
|
@ -38,8 +36,6 @@ import org.apache.druid.metadata.MetadataSupervisorManager;
|
|||
import org.apache.druid.metadata.SQLMetadataConnector;
|
||||
import org.apache.druid.metadata.SQLMetadataRuleManager;
|
||||
import org.apache.druid.metadata.SQLMetadataRuleManagerProvider;
|
||||
import org.apache.druid.metadata.SQLMetadataSegmentPublisher;
|
||||
import org.apache.druid.metadata.SQLMetadataSegmentPublisherProvider;
|
||||
import org.apache.druid.metadata.SQLMetadataSupervisorManager;
|
||||
import org.apache.druid.metadata.SegmentsMetadataManager;
|
||||
import org.apache.druid.metadata.SegmentsMetadataManagerProvider;
|
||||
|
@ -76,8 +72,6 @@ public class SQLMetadataStorageDruidModule implements Module
|
|||
PolyBind.createChoiceWithDefault(binder, prop, Key.get(SegmentsMetadataManagerProvider.class), defaultValue);
|
||||
PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataRuleManager.class), defaultValue);
|
||||
PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataRuleManagerProvider.class), defaultValue);
|
||||
PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataSegmentPublisher.class), defaultValue);
|
||||
PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataSegmentPublisherProvider.class), defaultValue);
|
||||
PolyBind.createChoiceWithDefault(binder, prop, Key.get(IndexerMetadataStorageCoordinator.class), defaultValue);
|
||||
PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataStorageActionHandlerFactory.class), defaultValue);
|
||||
PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataStorageUpdaterJobHandler.class), defaultValue);
|
||||
|
@ -109,16 +103,6 @@ public class SQLMetadataStorageDruidModule implements Module
|
|||
.to(SQLMetadataRuleManagerProvider.class)
|
||||
.in(LazySingleton.class);
|
||||
|
||||
PolyBind.optionBinder(binder, Key.get(MetadataSegmentPublisher.class))
|
||||
.addBinding(type)
|
||||
.to(SQLMetadataSegmentPublisher.class)
|
||||
.in(LazySingleton.class);
|
||||
|
||||
PolyBind.optionBinder(binder, Key.get(MetadataSegmentPublisherProvider.class))
|
||||
.addBinding(type)
|
||||
.to(SQLMetadataSegmentPublisherProvider.class)
|
||||
.in(LazySingleton.class);
|
||||
|
||||
PolyBind.optionBinder(binder, Key.get(IndexerMetadataStorageCoordinator.class))
|
||||
.addBinding(type)
|
||||
.to(IndexerSQLMetadataStorageCoordinator.class)
|
||||
|
|
|
@ -204,7 +204,7 @@ public interface IndexerMetadataStorageCoordinator
|
|||
*
|
||||
* @return set of segments actually added
|
||||
*/
|
||||
Set<DataSegment> commitSegments(Set<DataSegment> segments, @Nullable SegmentSchemaMapping segmentSchemaMapping) throws IOException;
|
||||
Set<DataSegment> commitSegments(Set<DataSegment> segments, @Nullable SegmentSchemaMapping segmentSchemaMapping);
|
||||
|
||||
/**
|
||||
* Allocates pending segments for the given requests in the pending segments table.
|
||||
|
@ -316,7 +316,7 @@ public interface IndexerMetadataStorageCoordinator
|
|||
@Nullable DataSourceMetadata startMetadata,
|
||||
@Nullable DataSourceMetadata endMetadata,
|
||||
@Nullable SegmentSchemaMapping segmentSchemaMapping
|
||||
) throws IOException;
|
||||
);
|
||||
|
||||
/**
|
||||
* Commits segments and corresponding schema created by an APPEND task.
|
||||
|
|
|
@ -450,7 +450,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||
public Set<DataSegment> commitSegments(
|
||||
final Set<DataSegment> segments,
|
||||
@Nullable final SegmentSchemaMapping segmentSchemaMapping
|
||||
) throws IOException
|
||||
)
|
||||
{
|
||||
final SegmentPublishResult result =
|
||||
commitSegmentsAndMetadata(
|
||||
|
@ -474,7 +474,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||
@Nullable final DataSourceMetadata startMetadata,
|
||||
@Nullable final DataSourceMetadata endMetadata,
|
||||
@Nullable final SegmentSchemaMapping segmentSchemaMapping
|
||||
) throws IOException
|
||||
)
|
||||
{
|
||||
verifySegmentsToCommit(segments);
|
||||
|
||||
|
|
|
@ -1,26 +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.metadata;
|
||||
|
||||
public interface MetadataSegmentPublisherProvider extends SegmentPublisherProvider
|
||||
{
|
||||
@Override
|
||||
MetadataSegmentPublisher get();
|
||||
}
|
|
@ -1,126 +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.metadata;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.NoneShardSpec;
|
||||
import org.skife.jdbi.v2.DBI;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class SQLMetadataSegmentPublisher implements MetadataSegmentPublisher
|
||||
{
|
||||
private static final Logger log = new Logger(SQLMetadataSegmentPublisher.class);
|
||||
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final MetadataStorageTablesConfig config;
|
||||
private final SQLMetadataConnector connector;
|
||||
private final String statement;
|
||||
|
||||
@Inject
|
||||
public SQLMetadataSegmentPublisher(
|
||||
ObjectMapper jsonMapper,
|
||||
MetadataStorageTablesConfig config,
|
||||
SQLMetadataConnector connector
|
||||
)
|
||||
{
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.config = config;
|
||||
this.connector = connector;
|
||||
this.statement = StringUtils.format(
|
||||
"INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload, used_status_last_updated) "
|
||||
+ "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload, :used_status_last_updated)",
|
||||
config.getSegmentsTable(), connector.getQuoteString()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void publishSegment(final DataSegment segment) throws IOException
|
||||
{
|
||||
String now = DateTimes.nowUtc().toString();
|
||||
publishSegment(
|
||||
segment.getId().toString(),
|
||||
segment.getDataSource(),
|
||||
now,
|
||||
segment.getInterval().getStart().toString(),
|
||||
segment.getInterval().getEnd().toString(),
|
||||
(segment.getShardSpec() instanceof NoneShardSpec) ? false : true,
|
||||
segment.getVersion(),
|
||||
true,
|
||||
jsonMapper.writeValueAsBytes(segment),
|
||||
now
|
||||
);
|
||||
}
|
||||
|
||||
private void publishSegment(
|
||||
final String segmentId,
|
||||
final String dataSource,
|
||||
final String createdDate,
|
||||
final String start,
|
||||
final String end,
|
||||
final boolean partitioned,
|
||||
final String version,
|
||||
final boolean used,
|
||||
final byte[] payload,
|
||||
final String usedFlagLastUpdated
|
||||
)
|
||||
{
|
||||
try {
|
||||
final DBI dbi = connector.getDBI();
|
||||
List<Map<String, Object>> exists = dbi.withHandle(
|
||||
handle -> handle.createQuery(
|
||||
StringUtils.format("SELECT id FROM %s WHERE id=:id", config.getSegmentsTable())
|
||||
).bind("id", segmentId).list()
|
||||
);
|
||||
|
||||
if (!exists.isEmpty()) {
|
||||
log.info("Skipping publish of segment[%s] as it already exists in the metadata store.", segmentId);
|
||||
return;
|
||||
}
|
||||
|
||||
dbi.withHandle(
|
||||
handle ->
|
||||
handle.createStatement(statement)
|
||||
.bind("id", segmentId)
|
||||
.bind("dataSource", dataSource)
|
||||
.bind("created_date", createdDate)
|
||||
.bind("start", start)
|
||||
.bind("end", end)
|
||||
.bind("partitioned", partitioned)
|
||||
.bind("version", version)
|
||||
.bind("used", used)
|
||||
.bind("payload", payload)
|
||||
.bind("used_status_last_updated", usedFlagLastUpdated)
|
||||
.execute()
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.error(e, "Exception inserting into DB");
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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.metadata;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import javax.validation.constraints.NotNull;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class SQLMetadataSegmentPublisherProvider implements MetadataSegmentPublisherProvider
|
||||
{
|
||||
@JacksonInject
|
||||
@NotNull
|
||||
private SQLMetadataConnector connector = null;
|
||||
|
||||
@JacksonInject
|
||||
@NotNull
|
||||
private MetadataStorageTablesConfig config = null;
|
||||
|
||||
@JacksonInject
|
||||
@NotNull
|
||||
private ObjectMapper jsonMapper = null;
|
||||
|
||||
@Override
|
||||
public MetadataSegmentPublisher get()
|
||||
{
|
||||
return new SQLMetadataSegmentPublisher(jsonMapper, config, connector);
|
||||
}
|
||||
}
|
|
@ -1,36 +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.metadata;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import com.google.inject.Provider;
|
||||
import org.apache.druid.guice.NoopSegmentPublisherProvider;
|
||||
import org.apache.druid.segment.realtime.SegmentPublisher;
|
||||
|
||||
/**
|
||||
*/
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = NoopSegmentPublisherProvider.class)
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "metadata", value = MetadataSegmentPublisherProvider.class)
|
||||
})
|
||||
public interface SegmentPublisherProvider extends Provider<SegmentPublisher>
|
||||
{
|
||||
}
|
|
@ -17,12 +17,10 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.realtime.plumber;
|
||||
package org.apache.druid.segment.handoff;
|
||||
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.query.SegmentDescriptor;
|
||||
import org.apache.druid.segment.handoff.SegmentHandoffNotifier;
|
||||
import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory;
|
||||
|
||||
import java.util.concurrent.Executor;
|
||||
|
|
@ -19,15 +19,11 @@
|
|||
|
||||
package org.apache.druid.segment.indexing;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
|
||||
/**
|
||||
*/
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "realtime", value = RealtimeIOConfig.class)
|
||||
})
|
||||
public interface IOConfig
|
||||
{
|
||||
}
|
||||
|
|
|
@ -1,54 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.indexing;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.data.input.FirehoseFactory;
|
||||
import org.apache.druid.segment.realtime.plumber.PlumberSchool;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class RealtimeIOConfig implements IOConfig
|
||||
{
|
||||
private final FirehoseFactory firehoseFactory;
|
||||
private final PlumberSchool plumberSchool;
|
||||
|
||||
@JsonCreator
|
||||
public RealtimeIOConfig(
|
||||
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
|
||||
@JsonProperty("plumber") PlumberSchool plumberSchool
|
||||
)
|
||||
{
|
||||
this.firehoseFactory = firehoseFactory;
|
||||
this.plumberSchool = plumberSchool;
|
||||
}
|
||||
|
||||
@JsonProperty("firehose")
|
||||
public FirehoseFactory getFirehoseFactory()
|
||||
{
|
||||
return firehoseFactory;
|
||||
}
|
||||
|
||||
public PlumberSchool getPlumberSchool()
|
||||
{
|
||||
return plumberSchool;
|
||||
}
|
||||
}
|
|
@ -1,420 +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.indexing;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.incremental.AppendableIndexSpec;
|
||||
import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig;
|
||||
import org.apache.druid.segment.realtime.plumber.IntervalStartVersioningPolicy;
|
||||
import org.apache.druid.segment.realtime.plumber.RejectionPolicyFactory;
|
||||
import org.apache.druid.segment.realtime.plumber.ServerTimeRejectionPolicyFactory;
|
||||
import org.apache.druid.segment.realtime.plumber.VersioningPolicy;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
||||
import org.apache.druid.timeline.partition.ShardSpec;
|
||||
import org.joda.time.Period;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.time.Duration;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class RealtimeTuningConfig implements AppenderatorConfig
|
||||
{
|
||||
private static final Period DEFAULT_INTERMEDIATE_PERSIST_PERIOD = new Period("PT10M");
|
||||
private static final Period DEFAULT_WINDOW_PERIOD = new Period("PT10M");
|
||||
private static final VersioningPolicy DEFAULT_VERSIONING_POLICY = new IntervalStartVersioningPolicy();
|
||||
private static final RejectionPolicyFactory DEFAULT_REJECTION_POLICY_FACTORY = new ServerTimeRejectionPolicyFactory();
|
||||
private static final int DEFAULT_MAX_PENDING_PERSISTS = 0;
|
||||
private static final ShardSpec DEFAULT_SHARD_SPEC = new NumberedShardSpec(0, 1);
|
||||
private static final IndexSpec DEFAULT_INDEX_SPEC = IndexSpec.DEFAULT;
|
||||
private static final Boolean DEFAULT_REPORT_PARSE_EXCEPTIONS = Boolean.FALSE;
|
||||
private static final long DEFAULT_HANDOFF_CONDITION_TIMEOUT = Duration.ofMinutes(15).toMillis();
|
||||
private static final long DEFAULT_ALERT_TIMEOUT = 0;
|
||||
private static final String DEFAULT_DEDUP_COLUMN = null;
|
||||
|
||||
// Might make sense for this to be a builder
|
||||
public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File basePersistDirectory)
|
||||
{
|
||||
return new RealtimeTuningConfig(
|
||||
DEFAULT_APPENDABLE_INDEX,
|
||||
DEFAULT_MAX_ROWS_IN_MEMORY_REALTIME,
|
||||
0L,
|
||||
DEFAULT_SKIP_BYTES_IN_MEMORY_OVERHEAD_CHECK,
|
||||
DEFAULT_INTERMEDIATE_PERSIST_PERIOD,
|
||||
DEFAULT_WINDOW_PERIOD,
|
||||
basePersistDirectory,
|
||||
DEFAULT_VERSIONING_POLICY,
|
||||
DEFAULT_REJECTION_POLICY_FACTORY,
|
||||
DEFAULT_MAX_PENDING_PERSISTS,
|
||||
DEFAULT_SHARD_SPEC,
|
||||
DEFAULT_INDEX_SPEC,
|
||||
DEFAULT_INDEX_SPEC,
|
||||
0,
|
||||
0,
|
||||
DEFAULT_REPORT_PARSE_EXCEPTIONS,
|
||||
DEFAULT_HANDOFF_CONDITION_TIMEOUT,
|
||||
DEFAULT_ALERT_TIMEOUT,
|
||||
null,
|
||||
DEFAULT_DEDUP_COLUMN,
|
||||
DEFAULT_NUM_PERSIST_THREADS
|
||||
);
|
||||
}
|
||||
|
||||
private final AppendableIndexSpec appendableIndexSpec;
|
||||
private final int maxRowsInMemory;
|
||||
private final long maxBytesInMemory;
|
||||
private final boolean skipBytesInMemoryOverheadCheck;
|
||||
private final Period intermediatePersistPeriod;
|
||||
private final Period windowPeriod;
|
||||
private final File basePersistDirectory;
|
||||
private final VersioningPolicy versioningPolicy;
|
||||
private final RejectionPolicyFactory rejectionPolicyFactory;
|
||||
private final int maxPendingPersists;
|
||||
private final ShardSpec shardSpec;
|
||||
private final IndexSpec indexSpec;
|
||||
private final IndexSpec indexSpecForIntermediatePersists;
|
||||
private final int persistThreadPriority;
|
||||
private final int mergeThreadPriority;
|
||||
private final boolean reportParseExceptions;
|
||||
private final long handoffConditionTimeout;
|
||||
private final long alertTimeout;
|
||||
@Nullable
|
||||
private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory;
|
||||
@Nullable
|
||||
private final String dedupColumn;
|
||||
private final int numPersistThreads;
|
||||
|
||||
public RealtimeTuningConfig(
|
||||
@Nullable AppendableIndexSpec appendableIndexSpec,
|
||||
Integer maxRowsInMemory,
|
||||
Long maxBytesInMemory,
|
||||
@Nullable Boolean skipBytesInMemoryOverheadCheck,
|
||||
Period intermediatePersistPeriod,
|
||||
Period windowPeriod,
|
||||
File basePersistDirectory,
|
||||
VersioningPolicy versioningPolicy,
|
||||
RejectionPolicyFactory rejectionPolicyFactory,
|
||||
Integer maxPendingPersists,
|
||||
ShardSpec shardSpec,
|
||||
IndexSpec indexSpec,
|
||||
@Nullable IndexSpec indexSpecForIntermediatePersists,
|
||||
int persistThreadPriority,
|
||||
int mergeThreadPriority,
|
||||
Boolean reportParseExceptions,
|
||||
Long handoffConditionTimeout,
|
||||
Long alertTimeout,
|
||||
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
|
||||
@Nullable String dedupColumn,
|
||||
@Nullable Integer numPersistThreads
|
||||
)
|
||||
{
|
||||
this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec;
|
||||
this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY_REALTIME : maxRowsInMemory;
|
||||
// initializing this to 0, it will be lazily initialized to a value
|
||||
// @see #getMaxBytesInMemoryOrDefault()
|
||||
this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory;
|
||||
this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck == null ?
|
||||
DEFAULT_SKIP_BYTES_IN_MEMORY_OVERHEAD_CHECK : skipBytesInMemoryOverheadCheck;
|
||||
this.intermediatePersistPeriod = intermediatePersistPeriod == null
|
||||
? DEFAULT_INTERMEDIATE_PERSIST_PERIOD
|
||||
: intermediatePersistPeriod;
|
||||
this.windowPeriod = windowPeriod == null ? DEFAULT_WINDOW_PERIOD : windowPeriod;
|
||||
this.basePersistDirectory = basePersistDirectory;
|
||||
this.versioningPolicy = versioningPolicy;
|
||||
this.rejectionPolicyFactory = rejectionPolicyFactory == null
|
||||
? DEFAULT_REJECTION_POLICY_FACTORY
|
||||
: rejectionPolicyFactory;
|
||||
this.maxPendingPersists = maxPendingPersists == null ? DEFAULT_MAX_PENDING_PERSISTS : maxPendingPersists;
|
||||
this.shardSpec = shardSpec == null ? DEFAULT_SHARD_SPEC : shardSpec;
|
||||
this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec;
|
||||
this.indexSpecForIntermediatePersists = indexSpecForIntermediatePersists == null ?
|
||||
this.indexSpec : indexSpecForIntermediatePersists;
|
||||
this.mergeThreadPriority = mergeThreadPriority;
|
||||
this.persistThreadPriority = persistThreadPriority;
|
||||
this.reportParseExceptions = reportParseExceptions == null
|
||||
? DEFAULT_REPORT_PARSE_EXCEPTIONS
|
||||
: reportParseExceptions;
|
||||
this.handoffConditionTimeout = handoffConditionTimeout == null
|
||||
? DEFAULT_HANDOFF_CONDITION_TIMEOUT
|
||||
: handoffConditionTimeout;
|
||||
Preconditions.checkArgument(this.handoffConditionTimeout >= 0, "handoffConditionTimeout must be >= 0");
|
||||
|
||||
this.alertTimeout = alertTimeout == null ? DEFAULT_ALERT_TIMEOUT : alertTimeout;
|
||||
Preconditions.checkArgument(this.alertTimeout >= 0, "alertTimeout must be >= 0");
|
||||
this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory;
|
||||
this.dedupColumn = dedupColumn == null ? DEFAULT_DEDUP_COLUMN : dedupColumn;
|
||||
this.numPersistThreads = numPersistThreads == null ?
|
||||
DEFAULT_NUM_PERSIST_THREADS : Math.max(numPersistThreads, DEFAULT_NUM_PERSIST_THREADS);
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
private RealtimeTuningConfig(
|
||||
@JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec,
|
||||
@JsonProperty("maxRowsInMemory") Integer maxRowsInMemory,
|
||||
@JsonProperty("maxBytesInMemory") Long maxBytesInMemory,
|
||||
@JsonProperty("skipBytesInMemoryOverheadCheck") @Nullable Boolean skipBytesInMemoryOverheadCheck,
|
||||
@JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod,
|
||||
@JsonProperty("windowPeriod") Period windowPeriod,
|
||||
@JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory,
|
||||
@JsonProperty("maxPendingPersists") Integer maxPendingPersists,
|
||||
@JsonProperty("shardSpec") ShardSpec shardSpec,
|
||||
@JsonProperty("indexSpec") IndexSpec indexSpec,
|
||||
@JsonProperty("indexSpecForIntermediatePersists") @Nullable IndexSpec indexSpecForIntermediatePersists,
|
||||
@JsonProperty("persistThreadPriority") int persistThreadPriority,
|
||||
@JsonProperty("mergeThreadPriority") int mergeThreadPriority,
|
||||
@JsonProperty("reportParseExceptions") Boolean reportParseExceptions,
|
||||
@JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout,
|
||||
@JsonProperty("alertTimeout") Long alertTimeout,
|
||||
@JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
|
||||
@JsonProperty("dedupColumn") @Nullable String dedupColumn,
|
||||
@JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads
|
||||
)
|
||||
{
|
||||
this(
|
||||
appendableIndexSpec,
|
||||
maxRowsInMemory,
|
||||
maxBytesInMemory,
|
||||
skipBytesInMemoryOverheadCheck,
|
||||
intermediatePersistPeriod,
|
||||
windowPeriod,
|
||||
null,
|
||||
null,
|
||||
rejectionPolicyFactory,
|
||||
maxPendingPersists,
|
||||
shardSpec,
|
||||
indexSpec,
|
||||
indexSpecForIntermediatePersists,
|
||||
persistThreadPriority,
|
||||
mergeThreadPriority,
|
||||
reportParseExceptions,
|
||||
handoffConditionTimeout,
|
||||
alertTimeout,
|
||||
segmentWriteOutMediumFactory,
|
||||
dedupColumn,
|
||||
numPersistThreads
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public AppendableIndexSpec getAppendableIndexSpec()
|
||||
{
|
||||
return appendableIndexSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public int getMaxRowsInMemory()
|
||||
{
|
||||
return maxRowsInMemory;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public long getMaxBytesInMemory()
|
||||
{
|
||||
return maxBytesInMemory;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public boolean isSkipBytesInMemoryOverheadCheck()
|
||||
{
|
||||
return skipBytesInMemoryOverheadCheck;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public Period getIntermediatePersistPeriod()
|
||||
{
|
||||
return intermediatePersistPeriod;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Period getWindowPeriod()
|
||||
{
|
||||
return windowPeriod;
|
||||
}
|
||||
|
||||
@Override
|
||||
public File getBasePersistDirectory()
|
||||
{
|
||||
return Preconditions.checkNotNull(basePersistDirectory, "basePersistDirectory not set");
|
||||
}
|
||||
|
||||
public VersioningPolicy getVersioningPolicy()
|
||||
{
|
||||
return Preconditions.checkNotNull(versioningPolicy, "versioningPolicy not set");
|
||||
}
|
||||
|
||||
@JsonProperty("rejectionPolicy")
|
||||
public RejectionPolicyFactory getRejectionPolicyFactory()
|
||||
{
|
||||
return rejectionPolicyFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public int getMaxPendingPersists()
|
||||
{
|
||||
return maxPendingPersists;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PartitionsSpec getPartitionsSpec()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public ShardSpec getShardSpec()
|
||||
{
|
||||
return shardSpec;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public IndexSpec getIndexSpec()
|
||||
{
|
||||
return indexSpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public IndexSpec getIndexSpecForIntermediatePersists()
|
||||
{
|
||||
return indexSpecForIntermediatePersists;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getPersistThreadPriority()
|
||||
{
|
||||
return this.persistThreadPriority;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getMergeThreadPriority()
|
||||
{
|
||||
return this.mergeThreadPriority;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public boolean isReportParseExceptions()
|
||||
{
|
||||
return reportParseExceptions;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getHandoffConditionTimeout()
|
||||
{
|
||||
return handoffConditionTimeout;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public long getAlertTimeout()
|
||||
{
|
||||
return alertTimeout;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
@Nullable
|
||||
public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory()
|
||||
{
|
||||
return segmentWriteOutMediumFactory;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Nullable
|
||||
public String getDedupColumn()
|
||||
{
|
||||
return dedupColumn;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public int getNumPersistThreads()
|
||||
{
|
||||
return numPersistThreads;
|
||||
}
|
||||
|
||||
public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy)
|
||||
{
|
||||
return new RealtimeTuningConfig(
|
||||
appendableIndexSpec,
|
||||
maxRowsInMemory,
|
||||
maxBytesInMemory,
|
||||
skipBytesInMemoryOverheadCheck,
|
||||
intermediatePersistPeriod,
|
||||
windowPeriod,
|
||||
basePersistDirectory,
|
||||
policy,
|
||||
rejectionPolicyFactory,
|
||||
maxPendingPersists,
|
||||
shardSpec,
|
||||
indexSpec,
|
||||
indexSpecForIntermediatePersists,
|
||||
persistThreadPriority,
|
||||
mergeThreadPriority,
|
||||
reportParseExceptions,
|
||||
handoffConditionTimeout,
|
||||
alertTimeout,
|
||||
segmentWriteOutMediumFactory,
|
||||
dedupColumn,
|
||||
numPersistThreads
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RealtimeTuningConfig withBasePersistDirectory(File dir)
|
||||
{
|
||||
return new RealtimeTuningConfig(
|
||||
appendableIndexSpec,
|
||||
maxRowsInMemory,
|
||||
maxBytesInMemory,
|
||||
skipBytesInMemoryOverheadCheck,
|
||||
intermediatePersistPeriod,
|
||||
windowPeriod,
|
||||
dir,
|
||||
versioningPolicy,
|
||||
rejectionPolicyFactory,
|
||||
maxPendingPersists,
|
||||
shardSpec,
|
||||
indexSpec,
|
||||
indexSpecForIntermediatePersists,
|
||||
persistThreadPriority,
|
||||
mergeThreadPriority,
|
||||
reportParseExceptions,
|
||||
handoffConditionTimeout,
|
||||
alertTimeout,
|
||||
segmentWriteOutMediumFactory,
|
||||
dedupColumn,
|
||||
numPersistThreads
|
||||
);
|
||||
}
|
||||
}
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.segment.indexing;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
|
@ -29,9 +28,6 @@ import org.apache.druid.segment.incremental.OnheapIncrementalIndex;
|
|||
/**
|
||||
*/
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "realtime", value = RealtimeTuningConfig.class)
|
||||
})
|
||||
public interface TuningConfig
|
||||
{
|
||||
boolean DEFAULT_LOG_PARSE_EXCEPTIONS = false;
|
||||
|
|
|
@ -1,100 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.realtime;
|
||||
|
||||
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.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.IngestionSpec;
|
||||
import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
||||
import org.apache.druid.segment.indexing.RealtimeTuningConfig;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* A Fire Department has a Firehose and a Plumber.
|
||||
* <p>
|
||||
* This is a metaphor for a realtime stream (Firehose) and a coordinator of sinks (Plumber). The Firehose provides the
|
||||
* realtime stream of data. The Plumber directs each drop of water from the firehose into the correct sink and makes
|
||||
* sure that the sinks don't overflow.
|
||||
*/
|
||||
public class FireDepartment extends IngestionSpec<RealtimeIOConfig, RealtimeTuningConfig>
|
||||
{
|
||||
private final DataSchema dataSchema;
|
||||
private final RealtimeIOConfig ioConfig;
|
||||
private final RealtimeTuningConfig tuningConfig;
|
||||
private final FireDepartmentMetrics metrics = new FireDepartmentMetrics();
|
||||
|
||||
@JsonCreator
|
||||
public FireDepartment(
|
||||
@JsonProperty("dataSchema") DataSchema dataSchema,
|
||||
@JsonProperty("ioConfig") RealtimeIOConfig ioConfig,
|
||||
@JsonProperty("tuningConfig") RealtimeTuningConfig tuningConfig
|
||||
)
|
||||
{
|
||||
super(dataSchema, ioConfig, tuningConfig);
|
||||
Preconditions.checkNotNull(dataSchema, "dataSchema");
|
||||
Preconditions.checkNotNull(ioConfig, "ioConfig");
|
||||
|
||||
this.dataSchema = dataSchema;
|
||||
this.ioConfig = ioConfig;
|
||||
this.tuningConfig = tuningConfig == null ? RealtimeTuningConfig.makeDefaultTuningConfig(null) : tuningConfig;
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Provides the data schema for the feed that this FireDepartment is in charge of.
|
||||
*
|
||||
* @return the Schema for this feed.
|
||||
*/
|
||||
@JsonProperty("dataSchema")
|
||||
@Override
|
||||
public DataSchema getDataSchema()
|
||||
{
|
||||
return dataSchema;
|
||||
}
|
||||
|
||||
@JsonProperty("ioConfig")
|
||||
@Override
|
||||
public RealtimeIOConfig getIOConfig()
|
||||
{
|
||||
return ioConfig;
|
||||
}
|
||||
|
||||
@JsonProperty("tuningConfig")
|
||||
@Override
|
||||
public RealtimeTuningConfig getTuningConfig()
|
||||
{
|
||||
return tuningConfig;
|
||||
}
|
||||
|
||||
public Firehose connect() throws IOException
|
||||
{
|
||||
return ioConfig.getFirehoseFactory()
|
||||
.connect(Preconditions.checkNotNull(dataSchema.getParser(), "inputRowParser"), null);
|
||||
}
|
||||
|
||||
public FireDepartmentMetrics getMetrics()
|
||||
{
|
||||
return metrics;
|
||||
}
|
||||
}
|
|
@ -1,58 +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;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.joda.time.Period;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class FireDepartmentConfig
|
||||
{
|
||||
private final int maxRowsInMemory;
|
||||
private final Period intermediatePersistPeriod;
|
||||
|
||||
@JsonCreator
|
||||
public FireDepartmentConfig(
|
||||
@JsonProperty("maxRowsInMemory") int maxRowsInMemory,
|
||||
@JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod
|
||||
)
|
||||
{
|
||||
this.maxRowsInMemory = maxRowsInMemory;
|
||||
this.intermediatePersistPeriod = intermediatePersistPeriod;
|
||||
|
||||
Preconditions.checkArgument(maxRowsInMemory > 0, "maxRowsInMemory[%s] should be greater than 0", maxRowsInMemory);
|
||||
Preconditions.checkNotNull(intermediatePersistPeriod, "intermediatePersistPeriod");
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getMaxRowsInMemory()
|
||||
{
|
||||
return maxRowsInMemory;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Period getIntermediatePersistPeriod()
|
||||
{
|
||||
return intermediatePersistPeriod;
|
||||
}
|
||||
}
|
|
@ -1,33 +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;
|
||||
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class NoopSegmentPublisher implements SegmentPublisher
|
||||
{
|
||||
@Override
|
||||
public void publishSegment(DataSegment segment)
|
||||
{
|
||||
// do nothing
|
||||
}
|
||||
}
|
|
@ -1,132 +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;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
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.MonitorUtils;
|
||||
import org.apache.druid.query.DruidMetrics;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* RealtimeMetricsMonitor is only used by RealtimeIndexTask, this monitor only supports FireDepartmentMetrics.
|
||||
* New ingestion task types should support RowIngestionMeters and use TaskRealtimeMetricsMonitor instead.
|
||||
* Please see the comment on RowIngestionMeters for more information regarding the relationship between
|
||||
* RowIngestionMeters and FireDepartmentMetrics.
|
||||
*/
|
||||
public class RealtimeMetricsMonitor extends AbstractMonitor
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(RealtimeMetricsMonitor.class);
|
||||
|
||||
private final Map<FireDepartment, FireDepartmentMetrics> previousValues;
|
||||
private final List<FireDepartment> fireDepartments;
|
||||
private final Map<String, String[]> dimensions;
|
||||
|
||||
@Inject
|
||||
public RealtimeMetricsMonitor(List<FireDepartment> fireDepartments)
|
||||
{
|
||||
this(fireDepartments, ImmutableMap.of());
|
||||
}
|
||||
|
||||
public RealtimeMetricsMonitor(List<FireDepartment> fireDepartments, Map<String, String[]> dimensions)
|
||||
{
|
||||
this.fireDepartments = fireDepartments;
|
||||
this.previousValues = new HashMap<>();
|
||||
this.dimensions = ImmutableMap.copyOf(dimensions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean doMonitor(ServiceEmitter emitter)
|
||||
{
|
||||
for (FireDepartment fireDepartment : fireDepartments) {
|
||||
FireDepartmentMetrics metrics = fireDepartment.getMetrics().snapshot();
|
||||
FireDepartmentMetrics previous = previousValues.get(fireDepartment);
|
||||
|
||||
if (previous == null) {
|
||||
previous = new FireDepartmentMetrics();
|
||||
}
|
||||
|
||||
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder()
|
||||
.setDimension(DruidMetrics.DATASOURCE, fireDepartment.getDataSchema().getDataSource());
|
||||
MonitorUtils.addDimensionsToBuilder(builder, dimensions);
|
||||
|
||||
final long thrownAway = metrics.thrownAway() - previous.thrownAway();
|
||||
if (thrownAway > 0) {
|
||||
log.warn(
|
||||
"[%,d] events thrown away. Possible causes: null events, events filtered out by transformSpec, or events outside windowPeriod.",
|
||||
thrownAway
|
||||
);
|
||||
}
|
||||
emitter.emit(builder.setMetric("ingest/events/thrownAway", thrownAway));
|
||||
final long unparseable = metrics.unparseable() - previous.unparseable();
|
||||
if (unparseable > 0) {
|
||||
log.error(
|
||||
"[%,d] unparseable events discarded. Turn on debug logging to see exception stack trace.",
|
||||
unparseable
|
||||
);
|
||||
}
|
||||
emitter.emit(builder.setMetric("ingest/events/unparseable", unparseable));
|
||||
final long dedup = metrics.dedup() - previous.dedup();
|
||||
if (dedup > 0) {
|
||||
log.warn("[%,d] duplicate events!", dedup);
|
||||
}
|
||||
emitter.emit(builder.setMetric("ingest/events/duplicate", dedup));
|
||||
|
||||
emitter.emit(builder.setMetric("ingest/events/processed", metrics.processed() - previous.processed()));
|
||||
emitter.emit(builder.setMetric("ingest/rows/output", metrics.rowOutput() - previous.rowOutput()));
|
||||
emitter.emit(builder.setMetric("ingest/persists/count", metrics.numPersists() - previous.numPersists()));
|
||||
emitter.emit(builder.setMetric("ingest/persists/time", metrics.persistTimeMillis() - previous.persistTimeMillis()));
|
||||
emitter.emit(builder.setMetric("ingest/persists/cpu", metrics.persistCpuTime() - previous.persistCpuTime()));
|
||||
emitter.emit(
|
||||
builder.setMetric(
|
||||
"ingest/persists/backPressure",
|
||||
metrics.persistBackPressureMillis() - previous.persistBackPressureMillis()
|
||||
)
|
||||
);
|
||||
emitter.emit(builder.setMetric("ingest/persists/failed", metrics.failedPersists() - previous.failedPersists()));
|
||||
emitter.emit(builder.setMetric("ingest/handoff/failed", metrics.failedHandoffs() - previous.failedHandoffs()));
|
||||
emitter.emit(builder.setMetric("ingest/merge/time", metrics.mergeTimeMillis() - previous.mergeTimeMillis()));
|
||||
emitter.emit(builder.setMetric("ingest/merge/cpu", metrics.mergeCpuTime() - previous.mergeCpuTime()));
|
||||
emitter.emit(builder.setMetric("ingest/handoff/count", metrics.handOffCount() - previous.handOffCount()));
|
||||
emitter.emit(builder.setMetric("ingest/sink/count", metrics.sinkCount()));
|
||||
|
||||
long messageGap = metrics.messageGap();
|
||||
if (messageGap >= 0) {
|
||||
emitter.emit(builder.setMetric("ingest/events/messageGap", messageGap));
|
||||
}
|
||||
|
||||
long maxSegmentHandoffTime = metrics.maxSegmentHandoffTime();
|
||||
if (maxSegmentHandoffTime >= 0) {
|
||||
emitter.emit(builder.setMetric("ingest/handoff/time", maxSegmentHandoffTime));
|
||||
}
|
||||
|
||||
previousValues.put(fireDepartment, metrics);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -25,17 +25,14 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
/**
|
||||
* Metrics for segment generation.
|
||||
*/
|
||||
public class FireDepartmentMetrics
|
||||
public class SegmentGenerationMetrics
|
||||
{
|
||||
private static final long NO_EMIT_SEGMENT_HANDOFF_TIME = -1L;
|
||||
|
||||
private static final long NO_EMIT_MESSAGE_GAP = -1L;
|
||||
|
||||
private final AtomicLong processedCount = new AtomicLong(0);
|
||||
private final AtomicLong processedWithErrorsCount = new AtomicLong(0);
|
||||
private final AtomicLong thrownAwayCount = new AtomicLong(0);
|
||||
private final AtomicLong unparseableCount = new AtomicLong(0);
|
||||
private final AtomicLong dedupCount = new AtomicLong(0);
|
||||
private final AtomicLong rowOutputCount = new AtomicLong(0);
|
||||
private final AtomicLong numPersists = new AtomicLong(0);
|
||||
|
@ -58,31 +55,6 @@ public class FireDepartmentMetrics
|
|||
|
||||
private final AtomicLong maxSegmentHandoffTime = new AtomicLong(NO_EMIT_SEGMENT_HANDOFF_TIME);
|
||||
|
||||
public void incrementProcessed()
|
||||
{
|
||||
processedCount.incrementAndGet();
|
||||
}
|
||||
|
||||
public void incrementProcessedWithErrors()
|
||||
{
|
||||
processedWithErrorsCount.incrementAndGet();
|
||||
}
|
||||
|
||||
public void incrementThrownAway()
|
||||
{
|
||||
thrownAwayCount.incrementAndGet();
|
||||
}
|
||||
|
||||
public void incrementDedup()
|
||||
{
|
||||
dedupCount.incrementAndGet();
|
||||
}
|
||||
|
||||
public void incrementUnparseable()
|
||||
{
|
||||
unparseableCount.incrementAndGet();
|
||||
}
|
||||
|
||||
public void incrementRowOutputCount(long numRows)
|
||||
{
|
||||
rowOutputCount.addAndGet(numRows);
|
||||
|
@ -113,11 +85,6 @@ public class FireDepartmentMetrics
|
|||
failedHandoffs.incrementAndGet();
|
||||
}
|
||||
|
||||
public void incrementMergeTimeMillis(long millis)
|
||||
{
|
||||
mergeTimeMillis.addAndGet(millis);
|
||||
}
|
||||
|
||||
public void incrementMergedRows(long rows)
|
||||
{
|
||||
mergedRows.addAndGet(rows);
|
||||
|
@ -128,16 +95,6 @@ public class FireDepartmentMetrics
|
|||
pushedRows.addAndGet(rows);
|
||||
}
|
||||
|
||||
public void incrementMergeCpuTime(long mergeTime)
|
||||
{
|
||||
mergeCpuTime.addAndGet(mergeTime);
|
||||
}
|
||||
|
||||
public void incrementPersistCpuTime(long persistTime)
|
||||
{
|
||||
persistCpuTime.addAndGet(persistTime);
|
||||
}
|
||||
|
||||
public void incrementHandOffCount()
|
||||
{
|
||||
handOffCount.incrementAndGet();
|
||||
|
@ -169,26 +126,6 @@ public class FireDepartmentMetrics
|
|||
return processingDone.get();
|
||||
}
|
||||
|
||||
public long processed()
|
||||
{
|
||||
return processedCount.get();
|
||||
}
|
||||
|
||||
public long processedWithErrors()
|
||||
{
|
||||
return processedWithErrorsCount.get();
|
||||
}
|
||||
|
||||
public long thrownAway()
|
||||
{
|
||||
return thrownAwayCount.get();
|
||||
}
|
||||
|
||||
public long unparseable()
|
||||
{
|
||||
return unparseableCount.get();
|
||||
}
|
||||
|
||||
public long dedup()
|
||||
{
|
||||
return dedupCount.get();
|
||||
|
@ -268,13 +205,9 @@ public class FireDepartmentMetrics
|
|||
return maxSegmentHandoffTime.get();
|
||||
}
|
||||
|
||||
public FireDepartmentMetrics snapshot()
|
||||
public SegmentGenerationMetrics snapshot()
|
||||
{
|
||||
final FireDepartmentMetrics retVal = new FireDepartmentMetrics();
|
||||
retVal.processedCount.set(processedCount.get());
|
||||
retVal.processedWithErrorsCount.set(processedWithErrorsCount.get());
|
||||
retVal.thrownAwayCount.set(thrownAwayCount.get());
|
||||
retVal.unparseableCount.set(unparseableCount.get());
|
||||
final SegmentGenerationMetrics retVal = new SegmentGenerationMetrics();
|
||||
retVal.dedupCount.set(dedupCount.get());
|
||||
retVal.rowOutputCount.set(rowOutputCount.get());
|
||||
retVal.numPersists.set(numPersists.get());
|
|
@ -1,29 +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;
|
||||
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public interface SegmentPublisher
|
||||
{
|
||||
void publishSegment(DataSegment segment) throws IOException;
|
||||
}
|
|
@ -1,40 +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.appenderator;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.RealtimeTuningConfig;
|
||||
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
|
||||
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "default", value = DefaultRealtimeAppenderatorFactory.class),
|
||||
@JsonSubTypes.Type(name = "offline", value = DefaultOfflineAppenderatorFactory.class)
|
||||
})
|
||||
public interface AppenderatorFactory
|
||||
{
|
||||
Appenderator build(
|
||||
DataSchema schema,
|
||||
RealtimeTuningConfig config,
|
||||
FireDepartmentMetrics metrics
|
||||
);
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue