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:
Clint Wylie 2024-06-24 20:13:33 -07:00 committed by GitHub
parent 2131917f16
commit 37a50e6803
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
176 changed files with 2218 additions and 14272 deletions

View File

@ -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

View File

@ -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')

View File

@ -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/).

View File

@ -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);

View File

@ -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;
}
);

View File

@ -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;

View File

@ -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;

View File

@ -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(

View File

@ -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);

View File

@ -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);

View File

@ -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;

View File

@ -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;

View File

@ -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)
);
}

View File

@ -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);
}
}

View File

@ -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
);
}
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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,

View File

@ -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,

View File

@ -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()
);
}
}
}

View File

@ -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)
})

View File

@ -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,

View File

@ -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);

View File

@ -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;

View File

@ -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(

View File

@ -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()

View File

@ -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

View File

@ -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;

View File

@ -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));
}

View File

@ -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();

View File

@ -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();

View File

@ -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 +
'}';
}
}
}
}

View File

@ -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;
}

View File

@ -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());

View File

@ -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;

View File

@ -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
{

View File

@ -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,

View File

@ -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
);
}
}

View File

@ -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

View File

@ -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

View 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

View File

@ -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

View File

@ -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>

View File

@ -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
{
}

View File

@ -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

View File

@ -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);
}
};
}
}

View File

@ -19,4 +19,4 @@
kafka:
instances:
- port: 9092
proxyPort: 9093
proxyPort: 9094

View File

@ -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"
}'
)

View File

@ -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,

View File

@ -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"
}
}
}

View File

@ -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"
}
}
}
}

View File

@ -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"
}
}
}
}

View File

@ -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"}
}
}

View File

@ -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"
}
}
]
}
]

View File

@ -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.
*/

View File

@ -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();
}

View File

@ -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

View File

@ -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;
}
}

View File

@ -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();
}
}
}

View File

@ -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();
}
}
}

View File

@ -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}

View File

@ -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,

View File

@ -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"
}
}
}

View File

@ -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}

View File

@ -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"
}
}
}
}

View File

@ -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"
}
}
}
}

View File

@ -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"
}
}
]
}
]

View File

@ -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();
}

View File

@ -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.

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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

View File

@ -97,10 +97,4 @@ public class IncrementalIndexAddResult
{
return reasonOfNotAdded == null && parseException == null;
}
@Nullable
public String getReasonOfNotAdded()
{
return reasonOfNotAdded;
}
}

View File

@ -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

View File

@ -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;

View File

@ -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

View File

@ -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();
}
}

View File

@ -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)

View File

@ -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.

View File

@ -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);

View File

@ -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();
}

View File

@ -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);
}
}
}

View File

@ -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);
}
}

View File

@ -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>
{
}

View File

@ -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;

View File

@ -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
{
}

View File

@ -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;
}
}

View File

@ -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
);
}
}

View File

@ -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;

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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
}
}

View File

@ -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;
}
}

View File

@ -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());

View File

@ -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;
}

View File

@ -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