mirror of
https://github.com/apache/druid.git
synced 2025-02-18 16:12:23 +00:00
Display row stats for multiphase parallel indexing tasks (#12280)
Row stats are reported for single phase tasks in the `/liveReports` and `/rowStats` APIs and are also a part of the overall task report. This commit adds changes to report row stats for multiphase tasks too. Changes: - Add `TaskReport` in `GeneratedPartitionsReport` generated during hash and range partitioning - Collect the reports for `index_generate` phase in `ParallelIndexSupervisorTask`
This commit is contained in:
parent
50038d9344
commit
1af4c9c933
@ -21,8 +21,10 @@ package org.apache.druid.indexing.common.task.batch.parallel;
|
|||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import org.apache.druid.indexing.common.TaskReport;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Report containing the {@link GenericPartitionStat}s created by a {@link PartialSegmentGenerateTask}. This report is
|
* Report containing the {@link GenericPartitionStat}s created by a {@link PartialSegmentGenerateTask}. This report is
|
||||||
@ -35,9 +37,10 @@ class GeneratedPartitionsMetadataReport extends GeneratedPartitionsReport
|
|||||||
@JsonCreator
|
@JsonCreator
|
||||||
GeneratedPartitionsMetadataReport(
|
GeneratedPartitionsMetadataReport(
|
||||||
@JsonProperty("taskId") String taskId,
|
@JsonProperty("taskId") String taskId,
|
||||||
@JsonProperty("partitionStats") List<PartitionStat> partitionStats
|
@JsonProperty("partitionStats") List<PartitionStat> partitionStats,
|
||||||
|
@JsonProperty("taskReport") Map<String, TaskReport> taskReport
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(taskId, partitionStats);
|
super(taskId, partitionStats, taskReport);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -20,8 +20,10 @@
|
|||||||
package org.apache.druid.indexing.common.task.batch.parallel;
|
package org.apache.druid.indexing.common.task.batch.parallel;
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import org.apache.druid.indexing.common.TaskReport;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -33,11 +35,13 @@ public class GeneratedPartitionsReport implements SubTaskReport
|
|||||||
{
|
{
|
||||||
private final String taskId;
|
private final String taskId;
|
||||||
private final List<PartitionStat> partitionStats;
|
private final List<PartitionStat> partitionStats;
|
||||||
|
private final Map<String, TaskReport> taskReport;
|
||||||
|
|
||||||
GeneratedPartitionsReport(String taskId, List<PartitionStat> partitionStats)
|
GeneratedPartitionsReport(String taskId, List<PartitionStat> partitionStats, Map<String, TaskReport> taskReport)
|
||||||
{
|
{
|
||||||
this.taskId = taskId;
|
this.taskId = taskId;
|
||||||
this.partitionStats = partitionStats;
|
this.partitionStats = partitionStats;
|
||||||
|
this.taskReport = taskReport;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -47,6 +51,12 @@ public class GeneratedPartitionsReport implements SubTaskReport
|
|||||||
return taskId;
|
return taskId;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public Map<String, TaskReport> getTaskReport()
|
||||||
|
{
|
||||||
|
return taskReport;
|
||||||
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public List<PartitionStat> getPartitionStats()
|
public List<PartitionStat> getPartitionStats()
|
||||||
{
|
{
|
||||||
@ -64,13 +74,14 @@ public class GeneratedPartitionsReport implements SubTaskReport
|
|||||||
}
|
}
|
||||||
GeneratedPartitionsReport that = (GeneratedPartitionsReport) o;
|
GeneratedPartitionsReport that = (GeneratedPartitionsReport) o;
|
||||||
return Objects.equals(taskId, that.taskId) &&
|
return Objects.equals(taskId, that.taskId) &&
|
||||||
Objects.equals(partitionStats, that.partitionStats);
|
Objects.equals(partitionStats, that.partitionStats) &&
|
||||||
|
Objects.equals(taskReport, that.taskReport);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode()
|
public int hashCode()
|
||||||
{
|
{
|
||||||
return Objects.hash(taskId, partitionStats);
|
return Objects.hash(taskId, partitionStats, taskReport);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -79,6 +90,7 @@ public class GeneratedPartitionsReport implements SubTaskReport
|
|||||||
return "GeneratedPartitionsReport{" +
|
return "GeneratedPartitionsReport{" +
|
||||||
"taskId='" + taskId + '\'' +
|
"taskId='" + taskId + '\'' +
|
||||||
", partitionStats=" + partitionStats +
|
", partitionStats=" + partitionStats +
|
||||||
|
", taskReport=" + taskReport +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -66,6 +66,7 @@ import org.apache.druid.java.util.common.Pair;
|
|||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularity;
|
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
|
import org.apache.druid.segment.incremental.MutableRowIngestionMeters;
|
||||||
import org.apache.druid.segment.incremental.ParseExceptionReport;
|
import org.apache.druid.segment.incremental.ParseExceptionReport;
|
||||||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||||
import org.apache.druid.segment.incremental.RowIngestionMetersTotals;
|
import org.apache.druid.segment.incremental.RowIngestionMetersTotals;
|
||||||
@ -176,6 +177,9 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
|
|||||||
@MonotonicNonNull
|
@MonotonicNonNull
|
||||||
private volatile TaskToolbox toolbox;
|
private volatile TaskToolbox toolbox;
|
||||||
|
|
||||||
|
@MonotonicNonNull
|
||||||
|
private Pair<Map<String, Object>, Map<String, Object>> indexGenerateRowStats;
|
||||||
|
|
||||||
private IngestionState ingestionState;
|
private IngestionState ingestionState;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
@ -726,6 +730,7 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
|
|||||||
);
|
);
|
||||||
return TaskStatus.failure(getId(), errMsg);
|
return TaskStatus.failure(getId(), errMsg);
|
||||||
}
|
}
|
||||||
|
indexGenerateRowStats = doGetRowStatsAndUnparseableEventsParallelMultiPhase(indexingRunner, true);
|
||||||
|
|
||||||
// 2. Partial segment merge phase
|
// 2. Partial segment merge phase
|
||||||
// partition (interval, partitionId) -> partition locations
|
// partition (interval, partitionId) -> partition locations
|
||||||
@ -814,6 +819,8 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
|
|||||||
return TaskStatus.failure(getId(), errMsg);
|
return TaskStatus.failure(getId(), errMsg);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
indexGenerateRowStats = doGetRowStatsAndUnparseableEventsParallelMultiPhase(indexingRunner, true);
|
||||||
|
|
||||||
// partition (interval, partitionId) -> partition locations
|
// partition (interval, partitionId) -> partition locations
|
||||||
Map<Partition, List<PartitionLocation>> partitionToLocations =
|
Map<Partition, List<PartitionLocation>> partitionToLocations =
|
||||||
getPartitionToLocations(indexingRunner.getReports());
|
getPartitionToLocations(indexingRunner.getReports());
|
||||||
@ -1477,10 +1484,7 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
|
|||||||
boolean includeUnparseable
|
boolean includeUnparseable
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
long processed = 0L;
|
final MutableRowIngestionMeters buildSegmentsRowStats = new MutableRowIngestionMeters();
|
||||||
long processedWithError = 0L;
|
|
||||||
long thrownAway = 0L;
|
|
||||||
long unparseable = 0L;
|
|
||||||
|
|
||||||
List<ParseExceptionReport> unparseableEvents = new ArrayList<>();
|
List<ParseExceptionReport> unparseableEvents = new ArrayList<>();
|
||||||
|
|
||||||
@ -1492,28 +1496,67 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
|
|||||||
LOG.warn("Got an empty task report from subtask: " + pushedSegmentsReport.getTaskId());
|
LOG.warn("Got an empty task report from subtask: " + pushedSegmentsReport.getTaskId());
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
IngestionStatsAndErrorsTaskReport ingestionStatsAndErrorsReport = (IngestionStatsAndErrorsTaskReport) taskReport.get(
|
RowIngestionMetersTotals rowIngestionMetersTotals =
|
||||||
IngestionStatsAndErrorsTaskReport.REPORT_KEY);
|
getBuildSegmentsStatsFromTaskReport(taskReport, includeUnparseable, unparseableEvents);
|
||||||
IngestionStatsAndErrorsTaskReportData reportData =
|
|
||||||
(IngestionStatsAndErrorsTaskReportData) ingestionStatsAndErrorsReport.getPayload();
|
buildSegmentsRowStats.addRowIngestionMetersTotals(rowIngestionMetersTotals);
|
||||||
RowIngestionMetersTotals totals = getTotalsFromBuildSegmentsRowStats(
|
}
|
||||||
reportData.getRowStats().get(RowIngestionMeters.BUILD_SEGMENTS)
|
|
||||||
|
RowIngestionMetersTotals rowStatsForRunningTasks = getRowStatsAndUnparseableEventsForRunningTasks(
|
||||||
|
parallelSinglePhaseRunner.getRunningTaskIds(),
|
||||||
|
unparseableEvents,
|
||||||
|
includeUnparseable
|
||||||
);
|
);
|
||||||
|
buildSegmentsRowStats.addRowIngestionMetersTotals(rowStatsForRunningTasks);
|
||||||
|
|
||||||
if (includeUnparseable) {
|
return createStatsAndErrorsReport(buildSegmentsRowStats.getTotals(), unparseableEvents);
|
||||||
List<ParseExceptionReport> taskUnparsebleEvents = (List<ParseExceptionReport>) reportData.getUnparseableEvents()
|
|
||||||
.get(RowIngestionMeters.BUILD_SEGMENTS);
|
|
||||||
unparseableEvents.addAll(taskUnparsebleEvents);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
processed += totals.getProcessed();
|
private Pair<Map<String, Object>, Map<String, Object>> doGetRowStatsAndUnparseableEventsParallelMultiPhase(
|
||||||
processedWithError += totals.getProcessedWithError();
|
ParallelIndexTaskRunner<?, ?> currentRunner,
|
||||||
thrownAway += totals.getThrownAway();
|
boolean includeUnparseable
|
||||||
unparseable += totals.getUnparseable();
|
)
|
||||||
|
{
|
||||||
|
if (indexGenerateRowStats != null) {
|
||||||
|
return Pair.of(indexGenerateRowStats.lhs, includeUnparseable ? indexGenerateRowStats.rhs : ImmutableMap.of());
|
||||||
|
} else if (!currentRunner.getName().equals("partial segment generation")) {
|
||||||
|
return Pair.of(ImmutableMap.of(), ImmutableMap.of());
|
||||||
|
} else {
|
||||||
|
Map<String, GeneratedPartitionsReport> completedSubtaskReports =
|
||||||
|
(Map<String, GeneratedPartitionsReport>) currentRunner.getReports();
|
||||||
|
|
||||||
|
final MutableRowIngestionMeters buildSegmentsRowStats = new MutableRowIngestionMeters();
|
||||||
|
final List<ParseExceptionReport> unparseableEvents = new ArrayList<>();
|
||||||
|
for (GeneratedPartitionsReport generatedPartitionsReport : completedSubtaskReports.values()) {
|
||||||
|
Map<String, TaskReport> taskReport = generatedPartitionsReport.getTaskReport();
|
||||||
|
if (taskReport == null || taskReport.isEmpty()) {
|
||||||
|
LOG.warn("Got an empty task report from subtask: " + generatedPartitionsReport.getTaskId());
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
RowIngestionMetersTotals rowStatsForCompletedTask =
|
||||||
|
getBuildSegmentsStatsFromTaskReport(taskReport, true, unparseableEvents);
|
||||||
|
|
||||||
|
buildSegmentsRowStats.addRowIngestionMetersTotals(rowStatsForCompletedTask);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Get stats from running tasks
|
RowIngestionMetersTotals rowStatsForRunningTasks = getRowStatsAndUnparseableEventsForRunningTasks(
|
||||||
Set<String> runningTaskIds = parallelSinglePhaseRunner.getRunningTaskIds();
|
currentRunner.getRunningTaskIds(),
|
||||||
|
unparseableEvents,
|
||||||
|
includeUnparseable
|
||||||
|
);
|
||||||
|
buildSegmentsRowStats.addRowIngestionMetersTotals(rowStatsForRunningTasks);
|
||||||
|
|
||||||
|
return createStatsAndErrorsReport(buildSegmentsRowStats.getTotals(), unparseableEvents);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private RowIngestionMetersTotals getRowStatsAndUnparseableEventsForRunningTasks(
|
||||||
|
Set<String> runningTaskIds,
|
||||||
|
List<ParseExceptionReport> unparseableEvents,
|
||||||
|
boolean includeUnparseable
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final MutableRowIngestionMeters buildSegmentsRowStats = new MutableRowIngestionMeters();
|
||||||
for (String runningTaskId : runningTaskIds) {
|
for (String runningTaskId : runningTaskIds) {
|
||||||
try {
|
try {
|
||||||
Map<String, Object> report = toolbox.getIndexingServiceClient().getTaskReport(runningTaskId);
|
Map<String, Object> report = toolbox.getIndexingServiceClient().getTaskReport(runningTaskId);
|
||||||
@ -1521,6 +1564,7 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
|
|||||||
// task does not have a running report yet
|
// task does not have a running report yet
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
Map<String, Object> ingestionStatsAndErrors = (Map<String, Object>) report.get("ingestionStatsAndErrors");
|
Map<String, Object> ingestionStatsAndErrors = (Map<String, Object>) report.get("ingestionStatsAndErrors");
|
||||||
Map<String, Object> payload = (Map<String, Object>) ingestionStatsAndErrors.get("payload");
|
Map<String, Object> payload = (Map<String, Object>) ingestionStatsAndErrors.get("payload");
|
||||||
Map<String, Object> rowStats = (Map<String, Object>) payload.get("rowStats");
|
Map<String, Object> rowStats = (Map<String, Object>) payload.get("rowStats");
|
||||||
@ -1529,33 +1573,53 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
|
|||||||
|
|
||||||
if (includeUnparseable) {
|
if (includeUnparseable) {
|
||||||
Map<String, Object> taskUnparseableEvents = (Map<String, Object>) payload.get("unparseableEvents");
|
Map<String, Object> taskUnparseableEvents = (Map<String, Object>) payload.get("unparseableEvents");
|
||||||
List<ParseExceptionReport> buildSegmentsUnparseableEvents = (List<ParseExceptionReport>) taskUnparseableEvents.get(
|
List<ParseExceptionReport> buildSegmentsUnparseableEvents = (List<ParseExceptionReport>)
|
||||||
RowIngestionMeters.BUILD_SEGMENTS
|
taskUnparseableEvents.get(RowIngestionMeters.BUILD_SEGMENTS);
|
||||||
);
|
|
||||||
unparseableEvents.addAll(buildSegmentsUnparseableEvents);
|
unparseableEvents.addAll(buildSegmentsUnparseableEvents);
|
||||||
}
|
}
|
||||||
|
|
||||||
processed += ((Number) buildSegments.get("processed")).longValue();
|
buildSegmentsRowStats.addRowIngestionMetersTotals(getTotalsFromBuildSegmentsRowStats(buildSegments));
|
||||||
processedWithError += ((Number) buildSegments.get("processedWithError")).longValue();
|
|
||||||
thrownAway += ((Number) buildSegments.get("thrownAway")).longValue();
|
|
||||||
unparseable += ((Number) buildSegments.get("unparseable")).longValue();
|
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
LOG.warn(e, "Encountered exception when getting live subtask report for task: " + runningTaskId);
|
LOG.warn(e, "Encountered exception when getting live subtask report for task: " + runningTaskId);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
return buildSegmentsRowStats.getTotals();
|
||||||
|
}
|
||||||
|
|
||||||
|
private Pair<Map<String, Object>, Map<String, Object>> createStatsAndErrorsReport(
|
||||||
|
RowIngestionMetersTotals rowStats,
|
||||||
|
List<ParseExceptionReport> unparseableEvents
|
||||||
|
)
|
||||||
|
{
|
||||||
Map<String, Object> rowStatsMap = new HashMap<>();
|
Map<String, Object> rowStatsMap = new HashMap<>();
|
||||||
Map<String, Object> totalsMap = new HashMap<>();
|
Map<String, Object> totalsMap = new HashMap<>();
|
||||||
totalsMap.put(
|
totalsMap.put(RowIngestionMeters.BUILD_SEGMENTS, rowStats);
|
||||||
RowIngestionMeters.BUILD_SEGMENTS,
|
|
||||||
new RowIngestionMetersTotals(processed, processedWithError, thrownAway, unparseable)
|
|
||||||
);
|
|
||||||
rowStatsMap.put("totals", totalsMap);
|
rowStatsMap.put("totals", totalsMap);
|
||||||
|
|
||||||
return Pair.of(rowStatsMap, ImmutableMap.of(RowIngestionMeters.BUILD_SEGMENTS, unparseableEvents));
|
return Pair.of(rowStatsMap, ImmutableMap.of(RowIngestionMeters.BUILD_SEGMENTS, unparseableEvents));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private RowIngestionMetersTotals getBuildSegmentsStatsFromTaskReport(
|
||||||
|
Map<String, TaskReport> taskReport,
|
||||||
|
boolean includeUnparseable,
|
||||||
|
List<ParseExceptionReport> unparseableEvents)
|
||||||
|
{
|
||||||
|
IngestionStatsAndErrorsTaskReport ingestionStatsAndErrorsReport = (IngestionStatsAndErrorsTaskReport) taskReport.get(
|
||||||
|
IngestionStatsAndErrorsTaskReport.REPORT_KEY);
|
||||||
|
IngestionStatsAndErrorsTaskReportData reportData =
|
||||||
|
(IngestionStatsAndErrorsTaskReportData) ingestionStatsAndErrorsReport.getPayload();
|
||||||
|
RowIngestionMetersTotals totals = getTotalsFromBuildSegmentsRowStats(
|
||||||
|
reportData.getRowStats().get(RowIngestionMeters.BUILD_SEGMENTS)
|
||||||
|
);
|
||||||
|
if (includeUnparseable) {
|
||||||
|
List<ParseExceptionReport> taskUnparsebleEvents = (List<ParseExceptionReport>) reportData.getUnparseableEvents()
|
||||||
|
.get(RowIngestionMeters.BUILD_SEGMENTS);
|
||||||
|
unparseableEvents.addAll(taskUnparsebleEvents);
|
||||||
|
}
|
||||||
|
return totals;
|
||||||
|
}
|
||||||
|
|
||||||
private Pair<Map<String, Object>, Map<String, Object>> doGetRowStatsAndUnparseableEvents(String full, boolean includeUnparseable)
|
private Pair<Map<String, Object>, Map<String, Object>> doGetRowStatsAndUnparseableEvents(String full, boolean includeUnparseable)
|
||||||
{
|
{
|
||||||
if (currentSubTaskHolder == null) {
|
if (currentSubTaskHolder == null) {
|
||||||
@ -1569,8 +1633,10 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
|
|||||||
|
|
||||||
if (isParallelMode()) {
|
if (isParallelMode()) {
|
||||||
if (isGuaranteedRollup(ingestionSchema.getIOConfig(), ingestionSchema.getTuningConfig())) {
|
if (isGuaranteedRollup(ingestionSchema.getIOConfig(), ingestionSchema.getTuningConfig())) {
|
||||||
// multiphase is not supported yet
|
return doGetRowStatsAndUnparseableEventsParallelMultiPhase(
|
||||||
return Pair.of(ImmutableMap.of(), ImmutableMap.of());
|
(ParallelIndexTaskRunner<?, ?>) currentRunner,
|
||||||
|
includeUnparseable
|
||||||
|
);
|
||||||
} else {
|
} else {
|
||||||
return doGetRowStatsAndUnparseableEventsParallelSinglePhase(
|
return doGetRowStatsAndUnparseableEventsParallelSinglePhase(
|
||||||
(SinglePhaseParallelIndexTaskRunner) currentRunner,
|
(SinglePhaseParallelIndexTaskRunner) currentRunner,
|
||||||
|
@ -22,6 +22,7 @@ package org.apache.druid.indexing.common.task.batch.parallel;
|
|||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
|
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
|
||||||
|
import org.apache.druid.indexing.common.TaskReport;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.actions.SurrogateTaskActionClient;
|
import org.apache.druid.indexing.common.actions.SurrogateTaskActionClient;
|
||||||
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||||
@ -161,12 +162,12 @@ public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask<G
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
GeneratedPartitionsMetadataReport createGeneratedPartitionsReport(TaskToolbox toolbox, List<DataSegment> segments)
|
GeneratedPartitionsMetadataReport createGeneratedPartitionsReport(TaskToolbox toolbox, List<DataSegment> segments, Map<String, TaskReport> taskReport)
|
||||||
{
|
{
|
||||||
List<PartitionStat> partitionStats = segments.stream()
|
List<PartitionStat> partitionStats = segments.stream()
|
||||||
.map(segment -> toolbox.getIntermediaryDataManager().generatePartitionStat(toolbox, segment))
|
.map(segment -> toolbox.getIntermediaryDataManager().generatePartitionStat(toolbox, segment))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
return new GeneratedPartitionsMetadataReport(getId(), partitionStats);
|
return new GeneratedPartitionsMetadataReport(getId(), partitionStats, taskReport);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -25,6 +25,7 @@ import com.google.common.base.Preconditions;
|
|||||||
import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
|
import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
|
||||||
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||||
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
||||||
|
import org.apache.druid.indexing.common.TaskReport;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.actions.SurrogateTaskActionClient;
|
import org.apache.druid.indexing.common.actions.SurrogateTaskActionClient;
|
||||||
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||||
@ -175,11 +176,11 @@ public class PartialRangeSegmentGenerateTask extends PartialSegmentGenerateTask<
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
GeneratedPartitionsMetadataReport createGeneratedPartitionsReport(TaskToolbox toolbox, List<DataSegment> segments)
|
GeneratedPartitionsMetadataReport createGeneratedPartitionsReport(TaskToolbox toolbox, List<DataSegment> segments, Map<String, TaskReport> taskReport)
|
||||||
{
|
{
|
||||||
List<PartitionStat> partitionStats = segments.stream()
|
List<PartitionStat> partitionStats = segments.stream()
|
||||||
.map(segment -> toolbox.getIntermediaryDataManager().generatePartitionStat(toolbox, segment))
|
.map(segment -> toolbox.getIntermediaryDataManager().generatePartitionStat(toolbox, segment))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
return new GeneratedPartitionsMetadataReport(getId(), partitionStats);
|
return new GeneratedPartitionsMetadataReport(getId(), partitionStats, taskReport);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -20,12 +20,18 @@
|
|||||||
package org.apache.druid.indexing.common.task.batch.parallel;
|
package org.apache.druid.indexing.common.task.batch.parallel;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
import org.apache.druid.data.input.InputSource;
|
import org.apache.druid.data.input.InputSource;
|
||||||
|
import org.apache.druid.indexer.IngestionState;
|
||||||
import org.apache.druid.indexer.TaskStatus;
|
import org.apache.druid.indexer.TaskStatus;
|
||||||
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||||
|
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport;
|
||||||
|
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
|
||||||
|
import org.apache.druid.indexing.common.TaskReport;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.task.BatchAppenderators;
|
import org.apache.druid.indexing.common.task.BatchAppenderators;
|
||||||
import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider;
|
import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider;
|
||||||
|
import org.apache.druid.indexing.common.task.IndexTaskUtils;
|
||||||
import org.apache.druid.indexing.common.task.InputSourceProcessor;
|
import org.apache.druid.indexing.common.task.InputSourceProcessor;
|
||||||
import org.apache.druid.indexing.common.task.SegmentAllocatorForBatch;
|
import org.apache.druid.indexing.common.task.SegmentAllocatorForBatch;
|
||||||
import org.apache.druid.indexing.common.task.SequenceNameFunction;
|
import org.apache.druid.indexing.common.task.SequenceNameFunction;
|
||||||
@ -35,6 +41,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.iterator.IndexTaskIn
|
|||||||
import org.apache.druid.indexing.worker.shuffle.ShuffleDataSegmentPusher;
|
import org.apache.druid.indexing.worker.shuffle.ShuffleDataSegmentPusher;
|
||||||
import org.apache.druid.query.DruidMetrics;
|
import org.apache.druid.query.DruidMetrics;
|
||||||
import org.apache.druid.segment.incremental.ParseExceptionHandler;
|
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.incremental.RowIngestionMeters;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
import org.apache.druid.segment.indexing.RealtimeIOConfig;
|
||||||
@ -46,10 +53,12 @@ import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
|
|||||||
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
|
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
|
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
|
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
@ -64,6 +73,12 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
|
|||||||
private final String supervisorTaskId;
|
private final String supervisorTaskId;
|
||||||
private final IndexTaskInputRowIteratorBuilder inputRowIteratorBuilder;
|
private final IndexTaskInputRowIteratorBuilder inputRowIteratorBuilder;
|
||||||
|
|
||||||
|
@MonotonicNonNull
|
||||||
|
private RowIngestionMeters buildSegmentsMeters;
|
||||||
|
|
||||||
|
@MonotonicNonNull
|
||||||
|
private ParseExceptionHandler parseExceptionHandler;
|
||||||
|
|
||||||
PartialSegmentGenerateTask(
|
PartialSegmentGenerateTask(
|
||||||
String id,
|
String id,
|
||||||
String groupId,
|
String groupId,
|
||||||
@ -113,7 +128,10 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
|
|||||||
inputSource,
|
inputSource,
|
||||||
toolbox.getIndexingTmpDir()
|
toolbox.getIndexingTmpDir()
|
||||||
);
|
);
|
||||||
taskClient.report(supervisorTaskId, createGeneratedPartitionsReport(toolbox, segments));
|
|
||||||
|
Map<String, TaskReport> taskReport = getTaskCompletionReports();
|
||||||
|
|
||||||
|
taskClient.report(supervisorTaskId, createGeneratedPartitionsReport(toolbox, segments, taskReport));
|
||||||
|
|
||||||
return TaskStatus.success(getId());
|
return TaskStatus.success(getId());
|
||||||
}
|
}
|
||||||
@ -131,7 +149,8 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
|
|||||||
*/
|
*/
|
||||||
abstract T createGeneratedPartitionsReport(
|
abstract T createGeneratedPartitionsReport(
|
||||||
TaskToolbox toolbox,
|
TaskToolbox toolbox,
|
||||||
List<DataSegment> segments
|
List<DataSegment> segments,
|
||||||
|
Map<String, TaskReport> taskReport
|
||||||
);
|
);
|
||||||
|
|
||||||
private List<DataSegment> generateSegments(
|
private List<DataSegment> generateSegments(
|
||||||
@ -148,7 +167,7 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
|
|||||||
null
|
null
|
||||||
);
|
);
|
||||||
final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
|
final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
|
||||||
final RowIngestionMeters buildSegmentsMeters = toolbox.getRowIngestionMetersFactory().createRowIngestionMeters();
|
buildSegmentsMeters = toolbox.getRowIngestionMetersFactory().createRowIngestionMeters();
|
||||||
|
|
||||||
toolbox.addMonitor(
|
toolbox.addMonitor(
|
||||||
new RealtimeMetricsMonitor(
|
new RealtimeMetricsMonitor(
|
||||||
@ -164,7 +183,7 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
|
|||||||
final SegmentAllocatorForBatch segmentAllocator = createSegmentAllocator(toolbox, taskClient);
|
final SegmentAllocatorForBatch segmentAllocator = createSegmentAllocator(toolbox, taskClient);
|
||||||
final SequenceNameFunction sequenceNameFunction = segmentAllocator.getSequenceNameFunction();
|
final SequenceNameFunction sequenceNameFunction = segmentAllocator.getSequenceNameFunction();
|
||||||
|
|
||||||
final ParseExceptionHandler parseExceptionHandler = new ParseExceptionHandler(
|
parseExceptionHandler = new ParseExceptionHandler(
|
||||||
buildSegmentsMeters,
|
buildSegmentsMeters,
|
||||||
tuningConfig.isLogParseExceptions(),
|
tuningConfig.isLogParseExceptions(),
|
||||||
tuningConfig.getMaxParseExceptions(),
|
tuningConfig.getMaxParseExceptions(),
|
||||||
@ -203,7 +222,6 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
|
|||||||
parseExceptionHandler,
|
parseExceptionHandler,
|
||||||
pushTimeout
|
pushTimeout
|
||||||
);
|
);
|
||||||
|
|
||||||
return pushed.getSegments();
|
return pushed.getSegments();
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
@ -218,4 +236,50 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Generate an IngestionStatsAndErrorsTaskReport for the task.
|
||||||
|
*/
|
||||||
|
private Map<String, TaskReport> getTaskCompletionReports()
|
||||||
|
{
|
||||||
|
return TaskReport.buildTaskReports(
|
||||||
|
new IngestionStatsAndErrorsTaskReport(
|
||||||
|
getId(),
|
||||||
|
new IngestionStatsAndErrorsTaskReportData(
|
||||||
|
IngestionState.COMPLETED,
|
||||||
|
getTaskCompletionUnparseableEvents(),
|
||||||
|
getTaskCompletionRowStats(),
|
||||||
|
"",
|
||||||
|
false, // not applicable for parallel subtask
|
||||||
|
segmentAvailabilityWaitTimeMs
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private Map<String, Object> getTaskCompletionUnparseableEvents()
|
||||||
|
{
|
||||||
|
Map<String, Object> unparseableEventsMap = new HashMap<>();
|
||||||
|
List<ParseExceptionReport> parseExceptionMessages = IndexTaskUtils.getReportListFromSavedParseExceptions(
|
||||||
|
parseExceptionHandler.getSavedParseExceptionReports()
|
||||||
|
);
|
||||||
|
|
||||||
|
if (parseExceptionMessages != null) {
|
||||||
|
unparseableEventsMap.put(RowIngestionMeters.BUILD_SEGMENTS, parseExceptionMessages);
|
||||||
|
} else {
|
||||||
|
unparseableEventsMap.put(RowIngestionMeters.BUILD_SEGMENTS, ImmutableList.of());
|
||||||
|
}
|
||||||
|
|
||||||
|
return unparseableEventsMap;
|
||||||
|
}
|
||||||
|
|
||||||
|
private Map<String, Object> getTaskCompletionRowStats()
|
||||||
|
{
|
||||||
|
Map<String, Object> metrics = new HashMap<>();
|
||||||
|
metrics.put(
|
||||||
|
RowIngestionMeters.BUILD_SEGMENTS,
|
||||||
|
buildSegmentsMeters.getTotals()
|
||||||
|
);
|
||||||
|
return metrics;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -149,7 +149,7 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
|
|||||||
boolean appendToExisting
|
boolean appendToExisting
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final ParallelIndexSupervisorTask task = newTask(
|
final ParallelIndexSupervisorTask task = createTask(
|
||||||
timestampSpec,
|
timestampSpec,
|
||||||
dimensionsSpec,
|
dimensionsSpec,
|
||||||
inputFormat,
|
inputFormat,
|
||||||
@ -165,15 +165,26 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
|
|||||||
return runTask(task, expectedTaskStatus);
|
return runTask(task, expectedTaskStatus);
|
||||||
}
|
}
|
||||||
|
|
||||||
Set<DataSegment> runTask(Task task, TaskState expectedTaskStatus)
|
void runTaskAndVerifyStatus(Task task, TaskState expectedTaskStatus)
|
||||||
{
|
{
|
||||||
task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK);
|
task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK);
|
||||||
TaskStatus taskStatus = getIndexingServiceClient().runAndWait(task);
|
TaskStatus taskStatus = getIndexingServiceClient().runAndWait(task);
|
||||||
Assert.assertEquals(expectedTaskStatus, taskStatus.getStatusCode());
|
Assert.assertEquals(expectedTaskStatus, taskStatus.getStatusCode());
|
||||||
|
}
|
||||||
|
|
||||||
|
Set<DataSegment> runTask(Task task, TaskState expectedTaskStatus)
|
||||||
|
{
|
||||||
|
runTaskAndVerifyStatus(task, expectedTaskStatus);
|
||||||
return getIndexingServiceClient().getPublishedSegments(task);
|
return getIndexingServiceClient().getPublishedSegments(task);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected ParallelIndexSupervisorTask newTask(
|
Map<String, Object> runTaskAndGetReports(Task task, TaskState expectedTaskStatus)
|
||||||
|
{
|
||||||
|
runTaskAndVerifyStatus(task, expectedTaskStatus);
|
||||||
|
return getIndexingServiceClient().getTaskReport(task.getId());
|
||||||
|
}
|
||||||
|
|
||||||
|
protected ParallelIndexSupervisorTask createTask(
|
||||||
@Nullable TimestampSpec timestampSpec,
|
@Nullable TimestampSpec timestampSpec,
|
||||||
@Nullable DimensionsSpec dimensionsSpec,
|
@Nullable DimensionsSpec dimensionsSpec,
|
||||||
@Nullable InputFormat inputFormat,
|
@Nullable InputFormat inputFormat,
|
||||||
|
@ -42,6 +42,7 @@ import org.apache.druid.data.input.impl.CsvInputFormat;
|
|||||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||||
import org.apache.druid.data.input.impl.ParseSpec;
|
import org.apache.druid.data.input.impl.ParseSpec;
|
||||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||||
|
import org.apache.druid.indexer.IngestionState;
|
||||||
import org.apache.druid.indexer.RunnerTaskState;
|
import org.apache.druid.indexer.RunnerTaskState;
|
||||||
import org.apache.druid.indexer.TaskLocation;
|
import org.apache.druid.indexer.TaskLocation;
|
||||||
import org.apache.druid.indexer.TaskStatus;
|
import org.apache.druid.indexer.TaskStatus;
|
||||||
@ -78,7 +79,9 @@ import org.apache.druid.math.expr.ExprMacroTable;
|
|||||||
import org.apache.druid.metadata.EntryExistsException;
|
import org.apache.druid.metadata.EntryExistsException;
|
||||||
import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable;
|
import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable;
|
||||||
import org.apache.druid.segment.IndexIO;
|
import org.apache.druid.segment.IndexIO;
|
||||||
|
import org.apache.druid.segment.incremental.ParseExceptionReport;
|
||||||
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
|
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
|
||||||
|
import org.apache.druid.segment.incremental.RowIngestionMetersTotals;
|
||||||
import org.apache.druid.segment.join.NoopJoinableFactory;
|
import org.apache.druid.segment.join.NoopJoinableFactory;
|
||||||
import org.apache.druid.segment.loading.LocalDataSegmentPuller;
|
import org.apache.druid.segment.loading.LocalDataSegmentPuller;
|
||||||
import org.apache.druid.segment.loading.LocalDataSegmentPusher;
|
import org.apache.druid.segment.loading.LocalDataSegmentPusher;
|
||||||
@ -101,6 +104,7 @@ import org.joda.time.DateTime;
|
|||||||
import org.joda.time.Duration;
|
import org.joda.time.Duration;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.rules.TemporaryFolder;
|
import org.junit.rules.TemporaryFolder;
|
||||||
@ -111,6 +115,7 @@ import java.io.IOException;
|
|||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
@ -125,6 +130,7 @@ import java.util.concurrent.ScheduledExecutorService;
|
|||||||
import java.util.concurrent.ThreadLocalRandom;
|
import java.util.concurrent.ThreadLocalRandom;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
|
public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
|
||||||
{
|
{
|
||||||
@ -297,7 +303,7 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
|
|||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
5,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null
|
null
|
||||||
@ -543,6 +549,16 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
|
|||||||
return taskRunner.run(injectIfNeeded(task));
|
return taskRunner.run(injectIfNeeded(task));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, Object> getTaskReport(String taskId)
|
||||||
|
{
|
||||||
|
final Optional<Task> task = getTaskStorage().getTask(taskId);
|
||||||
|
if (!task.isPresent()) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
return ((ParallelIndexSupervisorTask) task.get()).doGetLiveReports("full");
|
||||||
|
}
|
||||||
|
|
||||||
public TaskContainer getTaskContainer(String taskId)
|
public TaskContainer getTaskContainer(String taskId)
|
||||||
{
|
{
|
||||||
return taskRunner.getTaskContainer(taskId);
|
return taskRunner.getTaskContainer(taskId);
|
||||||
@ -791,6 +807,111 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected Map<String, Object> buildExpectedTaskReportSequential(
|
||||||
|
String taskId,
|
||||||
|
List<ParseExceptionReport> expectedUnparseableEvents,
|
||||||
|
RowIngestionMetersTotals expectedDeterminePartitions,
|
||||||
|
RowIngestionMetersTotals expectedTotals
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final Map<String, Object> payload = new HashMap<>();
|
||||||
|
|
||||||
|
payload.put("ingestionState", IngestionState.COMPLETED);
|
||||||
|
payload.put(
|
||||||
|
"unparseableEvents",
|
||||||
|
ImmutableMap.of("determinePartitions", ImmutableList.of(), "buildSegments", expectedUnparseableEvents)
|
||||||
|
);
|
||||||
|
Map<String, Object> emptyAverageMinuteMap = ImmutableMap.of(
|
||||||
|
"processed", 0.0,
|
||||||
|
"unparseable", 0.0,
|
||||||
|
"thrownAway", 0.0,
|
||||||
|
"processedWithError", 0.0
|
||||||
|
);
|
||||||
|
|
||||||
|
Map<String, Object> emptyAverages = ImmutableMap.of(
|
||||||
|
"1m", emptyAverageMinuteMap,
|
||||||
|
"5m", emptyAverageMinuteMap,
|
||||||
|
"15m", emptyAverageMinuteMap
|
||||||
|
);
|
||||||
|
|
||||||
|
payload.put(
|
||||||
|
"rowStats",
|
||||||
|
ImmutableMap.of(
|
||||||
|
"movingAverages",
|
||||||
|
ImmutableMap.of("determinePartitions", emptyAverages, "buildSegments", emptyAverages),
|
||||||
|
"totals",
|
||||||
|
ImmutableMap.of("determinePartitions", expectedDeterminePartitions, "buildSegments", expectedTotals)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
final Map<String, Object> ingestionStatsAndErrors = new HashMap<>();
|
||||||
|
ingestionStatsAndErrors.put("taskId", taskId);
|
||||||
|
ingestionStatsAndErrors.put("payload", payload);
|
||||||
|
ingestionStatsAndErrors.put("type", "ingestionStatsAndErrors");
|
||||||
|
|
||||||
|
return Collections.singletonMap("ingestionStatsAndErrors", ingestionStatsAndErrors);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected Map<String, Object> buildExpectedTaskReportParallel(
|
||||||
|
String taskId,
|
||||||
|
List<ParseExceptionReport> expectedUnparseableEvents,
|
||||||
|
RowIngestionMetersTotals expectedTotals
|
||||||
|
)
|
||||||
|
{
|
||||||
|
Map<String, Object> returnMap = new HashMap<>();
|
||||||
|
Map<String, Object> ingestionStatsAndErrors = new HashMap<>();
|
||||||
|
Map<String, Object> payload = new HashMap<>();
|
||||||
|
|
||||||
|
payload.put("ingestionState", IngestionState.COMPLETED);
|
||||||
|
payload.put("unparseableEvents", ImmutableMap.of("buildSegments", expectedUnparseableEvents));
|
||||||
|
payload.put("rowStats", ImmutableMap.of("totals", ImmutableMap.of("buildSegments", expectedTotals)));
|
||||||
|
|
||||||
|
ingestionStatsAndErrors.put("taskId", taskId);
|
||||||
|
ingestionStatsAndErrors.put("payload", payload);
|
||||||
|
ingestionStatsAndErrors.put("type", "ingestionStatsAndErrors");
|
||||||
|
|
||||||
|
returnMap.put("ingestionStatsAndErrors", ingestionStatsAndErrors);
|
||||||
|
return returnMap;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void compareTaskReports(
|
||||||
|
Map<String, Object> expectedReports,
|
||||||
|
Map<String, Object> actualReports
|
||||||
|
)
|
||||||
|
{
|
||||||
|
expectedReports = (Map<String, Object>) expectedReports.get("ingestionStatsAndErrors");
|
||||||
|
actualReports = (Map<String, Object>) actualReports.get("ingestionStatsAndErrors");
|
||||||
|
|
||||||
|
Assert.assertEquals(expectedReports.get("taskId"), actualReports.get("taskId"));
|
||||||
|
Assert.assertEquals(expectedReports.get("type"), actualReports.get("type"));
|
||||||
|
|
||||||
|
Map<String, Object> expectedPayload = (Map<String, Object>) expectedReports.get("payload");
|
||||||
|
Map<String, Object> actualPayload = (Map<String, Object>) actualReports.get("payload");
|
||||||
|
Assert.assertEquals(expectedPayload.get("ingestionState"), actualPayload.get("ingestionState"));
|
||||||
|
Assert.assertEquals(expectedPayload.get("rowStats"), actualPayload.get("rowStats"));
|
||||||
|
Assert.assertEquals(expectedPayload.get("ingestionState"), actualPayload.get("ingestionState"));
|
||||||
|
|
||||||
|
List<ParseExceptionReport> expectedParseExceptionReports =
|
||||||
|
(List<ParseExceptionReport>) ((Map<String, Object>)
|
||||||
|
expectedPayload.get("unparseableEvents")).get("buildSegments");
|
||||||
|
|
||||||
|
List<ParseExceptionReport> actualParseExceptionReports =
|
||||||
|
(List<ParseExceptionReport>) ((Map<String, Object>)
|
||||||
|
actualPayload.get("unparseableEvents")).get("buildSegments");
|
||||||
|
|
||||||
|
List<String> expectedMessages = expectedParseExceptionReports
|
||||||
|
.stream().map(r -> r.getDetails().get(0)).collect(Collectors.toList());
|
||||||
|
List<String> actualMessages = actualParseExceptionReports
|
||||||
|
.stream().map(r -> r.getDetails().get(0)).collect(Collectors.toList());
|
||||||
|
Assert.assertEquals(expectedMessages, actualMessages);
|
||||||
|
|
||||||
|
List<String> expectedInputs = expectedParseExceptionReports
|
||||||
|
.stream().map(ParseExceptionReport::getInput).collect(Collectors.toList());
|
||||||
|
List<String> actualInputs = actualParseExceptionReports
|
||||||
|
.stream().map(ParseExceptionReport::getInput).collect(Collectors.toList());
|
||||||
|
Assert.assertEquals(expectedInputs, actualInputs);
|
||||||
|
}
|
||||||
|
|
||||||
static class LocalParallelIndexTaskClientFactory implements IndexTaskClientFactory<ParallelIndexSupervisorTaskClient>
|
static class LocalParallelIndexTaskClientFactory implements IndexTaskClientFactory<ParallelIndexSupervisorTaskClient>
|
||||||
{
|
{
|
||||||
private final ConcurrentMap<String, TaskContainer> tasks;
|
private final ConcurrentMap<String, TaskContainer> tasks;
|
||||||
|
@ -36,6 +36,7 @@ import org.apache.druid.java.util.common.Intervals;
|
|||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.guava.Comparators;
|
import org.apache.druid.java.util.common.guava.Comparators;
|
||||||
import org.apache.druid.query.scan.ScanResultValue;
|
import org.apache.druid.query.scan.ScanResultValue;
|
||||||
|
import org.apache.druid.segment.incremental.RowIngestionMetersTotals;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
||||||
import org.apache.druid.timeline.partition.HashPartitionFunction;
|
import org.apache.druid.timeline.partition.HashPartitionFunction;
|
||||||
@ -161,15 +162,14 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
|
|||||||
public void testRun() throws Exception
|
public void testRun() throws Exception
|
||||||
{
|
{
|
||||||
final Integer maxRowsPerSegment = numShards == null ? 10 : null;
|
final Integer maxRowsPerSegment = numShards == null ? 10 : null;
|
||||||
final Set<DataSegment> publishedSegments = runTestTask(
|
final Set<DataSegment> publishedSegments = runTask(createTask(
|
||||||
new HashedPartitionsSpec(
|
new HashedPartitionsSpec(
|
||||||
maxRowsPerSegment,
|
maxRowsPerSegment,
|
||||||
numShards,
|
numShards,
|
||||||
ImmutableList.of("dim1", "dim2")
|
ImmutableList.of("dim1", "dim2")
|
||||||
),
|
),
|
||||||
TaskState.SUCCESS,
|
|
||||||
false
|
false
|
||||||
);
|
), TaskState.SUCCESS);
|
||||||
|
|
||||||
final Map<Interval, Integer> expectedIntervalToNumSegments = computeExpectedIntervalToNumSegments(
|
final Map<Interval, Integer> expectedIntervalToNumSegments = computeExpectedIntervalToNumSegments(
|
||||||
maxRowsPerSegment,
|
maxRowsPerSegment,
|
||||||
@ -182,16 +182,14 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
|
|||||||
public void testRunWithHashPartitionFunction() throws Exception
|
public void testRunWithHashPartitionFunction() throws Exception
|
||||||
{
|
{
|
||||||
final Integer maxRowsPerSegment = numShards == null ? 10 : null;
|
final Integer maxRowsPerSegment = numShards == null ? 10 : null;
|
||||||
final Set<DataSegment> publishedSegments = runTestTask(
|
final Set<DataSegment> publishedSegments = runTask(createTask(
|
||||||
new HashedPartitionsSpec(
|
new HashedPartitionsSpec(
|
||||||
maxRowsPerSegment,
|
maxRowsPerSegment,
|
||||||
numShards,
|
numShards,
|
||||||
ImmutableList.of("dim1", "dim2"),
|
ImmutableList.of("dim1", "dim2"),
|
||||||
HashPartitionFunction.MURMUR3_32_ABS
|
HashPartitionFunction.MURMUR3_32_ABS
|
||||||
),
|
),
|
||||||
TaskState.SUCCESS,
|
false), TaskState.SUCCESS);
|
||||||
false
|
|
||||||
);
|
|
||||||
final Map<Interval, Integer> expectedIntervalToNumSegments = computeExpectedIntervalToNumSegments(
|
final Map<Interval, Integer> expectedIntervalToNumSegments = computeExpectedIntervalToNumSegments(
|
||||||
maxRowsPerSegment,
|
maxRowsPerSegment,
|
||||||
numShards
|
numShards
|
||||||
@ -199,6 +197,39 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
|
|||||||
assertHashedPartition(publishedSegments, expectedIntervalToNumSegments);
|
assertHashedPartition(publishedSegments, expectedIntervalToNumSegments);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRowStats()
|
||||||
|
{
|
||||||
|
final Integer maxRowsPerSegment = numShards == null ? 10 : null;
|
||||||
|
ParallelIndexSupervisorTask task = createTask(
|
||||||
|
new HashedPartitionsSpec(
|
||||||
|
maxRowsPerSegment,
|
||||||
|
numShards,
|
||||||
|
ImmutableList.of("dim1", "dim2"),
|
||||||
|
HashPartitionFunction.MURMUR3_32_ABS),
|
||||||
|
false);
|
||||||
|
RowIngestionMetersTotals expectedTotals = new RowIngestionMetersTotals(200, 0, 0, 0);
|
||||||
|
Map<String, Object> expectedReports;
|
||||||
|
if (maxNumConcurrentSubTasks <= 1) {
|
||||||
|
expectedReports = buildExpectedTaskReportSequential(
|
||||||
|
task.getId(),
|
||||||
|
ImmutableList.of(),
|
||||||
|
numShards == null ? expectedTotals : new RowIngestionMetersTotals(0, 0, 0, 0),
|
||||||
|
expectedTotals
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
// when useInputFormatApi is false, maxConcurrentSubTasks=2 and it uses the single phase runner
|
||||||
|
// instead of sequential runner
|
||||||
|
expectedReports = buildExpectedTaskReportParallel(
|
||||||
|
task.getId(),
|
||||||
|
ImmutableList.of(),
|
||||||
|
expectedTotals
|
||||||
|
);
|
||||||
|
}
|
||||||
|
Map<String, Object> actualReports = runTaskAndGetReports(task, TaskState.SUCCESS);
|
||||||
|
compareTaskReports(expectedReports, actualReports);
|
||||||
|
}
|
||||||
|
|
||||||
private Map<Interval, Integer> computeExpectedIntervalToNumSegments(
|
private Map<Interval, Integer> computeExpectedIntervalToNumSegments(
|
||||||
@Nullable Integer maxRowsPerSegment,
|
@Nullable Integer maxRowsPerSegment,
|
||||||
@Nullable Integer numShards
|
@Nullable Integer numShards
|
||||||
@ -224,27 +255,26 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
|
|||||||
{
|
{
|
||||||
final Set<DataSegment> publishedSegments = new HashSet<>();
|
final Set<DataSegment> publishedSegments = new HashSet<>();
|
||||||
publishedSegments.addAll(
|
publishedSegments.addAll(
|
||||||
runTestTask(
|
runTask(
|
||||||
|
createTask(
|
||||||
new HashedPartitionsSpec(null, numShards, ImmutableList.of("dim1", "dim2")),
|
new HashedPartitionsSpec(null, numShards, ImmutableList.of("dim1", "dim2")),
|
||||||
TaskState.SUCCESS,
|
false),
|
||||||
false
|
TaskState.SUCCESS)
|
||||||
)
|
|
||||||
);
|
);
|
||||||
// Append
|
// Append
|
||||||
publishedSegments.addAll(
|
publishedSegments.addAll(
|
||||||
runTestTask(
|
runTask(
|
||||||
|
createTask(
|
||||||
new DynamicPartitionsSpec(5, null),
|
new DynamicPartitionsSpec(5, null),
|
||||||
TaskState.SUCCESS,
|
true),
|
||||||
true
|
TaskState.SUCCESS));
|
||||||
)
|
|
||||||
);
|
|
||||||
// And append again
|
// And append again
|
||||||
publishedSegments.addAll(
|
publishedSegments.addAll(
|
||||||
runTestTask(
|
runTask(
|
||||||
|
createTask(
|
||||||
new DynamicPartitionsSpec(10, null),
|
new DynamicPartitionsSpec(10, null),
|
||||||
TaskState.SUCCESS,
|
true),
|
||||||
true
|
TaskState.SUCCESS)
|
||||||
)
|
|
||||||
);
|
);
|
||||||
|
|
||||||
final Map<Interval, List<DataSegment>> intervalToSegments = new HashMap<>();
|
final Map<Interval, List<DataSegment>> intervalToSegments = new HashMap<>();
|
||||||
@ -275,14 +305,13 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private Set<DataSegment> runTestTask(
|
private ParallelIndexSupervisorTask createTask(
|
||||||
PartitionsSpec partitionsSpec,
|
PartitionsSpec partitionsSpec,
|
||||||
TaskState expectedTaskState,
|
|
||||||
boolean appendToExisting
|
boolean appendToExisting
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (isUseInputFormatApi()) {
|
if (isUseInputFormatApi()) {
|
||||||
return runTestTask(
|
return createTask(
|
||||||
TIMESTAMP_SPEC,
|
TIMESTAMP_SPEC,
|
||||||
DIMENSIONS_SPEC,
|
DIMENSIONS_SPEC,
|
||||||
INPUT_FORMAT,
|
INPUT_FORMAT,
|
||||||
@ -292,11 +321,10 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
|
|||||||
"test_*",
|
"test_*",
|
||||||
partitionsSpec,
|
partitionsSpec,
|
||||||
maxNumConcurrentSubTasks,
|
maxNumConcurrentSubTasks,
|
||||||
expectedTaskState,
|
|
||||||
appendToExisting
|
appendToExisting
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
return runTestTask(
|
return createTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
@ -306,7 +334,6 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
|
|||||||
"test_*",
|
"test_*",
|
||||||
partitionsSpec,
|
partitionsSpec,
|
||||||
maxNumConcurrentSubTasks,
|
maxNumConcurrentSubTasks,
|
||||||
expectedTaskState,
|
|
||||||
appendToExisting
|
appendToExisting
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -355,13 +355,13 @@ public class ParallelIndexSupervisorTaskTest
|
|||||||
createRangePartitionStat(day2, 7),
|
createRangePartitionStat(day2, 7),
|
||||||
createRangePartitionStat(day1, 0),
|
createRangePartitionStat(day1, 0),
|
||||||
createRangePartitionStat(day2, 1)
|
createRangePartitionStat(day2, 1)
|
||||||
)));
|
), null));
|
||||||
taskIdToReport.put(task2, new GeneratedPartitionsReport(task2, Arrays.asList(
|
taskIdToReport.put(task2, new GeneratedPartitionsReport(task2, Arrays.asList(
|
||||||
createRangePartitionStat(day1, 4),
|
createRangePartitionStat(day1, 4),
|
||||||
createRangePartitionStat(day1, 6),
|
createRangePartitionStat(day1, 6),
|
||||||
createRangePartitionStat(day2, 1),
|
createRangePartitionStat(day2, 1),
|
||||||
createRangePartitionStat(day1, 1)
|
createRangePartitionStat(day1, 1)
|
||||||
)));
|
), null));
|
||||||
|
|
||||||
Map<ParallelIndexSupervisorTask.Partition, List<PartitionLocation>> partitionToLocations
|
Map<ParallelIndexSupervisorTask.Partition, List<PartitionLocation>> partitionToLocations
|
||||||
= ParallelIndexSupervisorTask.getPartitionToLocations(taskIdToReport);
|
= ParallelIndexSupervisorTask.getPartitionToLocations(taskIdToReport);
|
||||||
|
@ -41,6 +41,7 @@ import org.apache.druid.indexing.common.LockGranularity;
|
|||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.guava.Comparators;
|
import org.apache.druid.java.util.common.guava.Comparators;
|
||||||
import org.apache.druid.query.scan.ScanResultValue;
|
import org.apache.druid.query.scan.ScanResultValue;
|
||||||
|
import org.apache.druid.segment.incremental.RowIngestionMetersTotals;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.apache.druid.timeline.partition.DimensionRangeShardSpec;
|
import org.apache.druid.timeline.partition.DimensionRangeShardSpec;
|
||||||
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
||||||
@ -225,22 +226,39 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
|
|||||||
public void createsCorrectRangePartitions() throws Exception
|
public void createsCorrectRangePartitions() throws Exception
|
||||||
{
|
{
|
||||||
int targetRowsPerSegment = NUM_ROW * 2 / DIM_FILE_CARDINALITY / NUM_PARTITION;
|
int targetRowsPerSegment = NUM_ROW * 2 / DIM_FILE_CARDINALITY / NUM_PARTITION;
|
||||||
final Set<DataSegment> publishedSegments = runTestTask(
|
final Set<DataSegment> publishedSegments = runTask(runTestTask(
|
||||||
new DimensionRangePartitionsSpec(
|
new DimensionRangePartitionsSpec(
|
||||||
targetRowsPerSegment,
|
targetRowsPerSegment,
|
||||||
null,
|
null,
|
||||||
Collections.singletonList(DIM1),
|
Collections.singletonList(DIM1),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
useMultivalueDim ? TaskState.FAILED : TaskState.SUCCESS,
|
|
||||||
false
|
false
|
||||||
);
|
), useMultivalueDim ? TaskState.FAILED : TaskState.SUCCESS);
|
||||||
|
|
||||||
if (!useMultivalueDim) {
|
if (!useMultivalueDim) {
|
||||||
assertRangePartitions(publishedSegments);
|
assertRangePartitions(publishedSegments);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRowStats()
|
||||||
|
{
|
||||||
|
if (useMultivalueDim) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
final int targetRowsPerSegment = NUM_ROW / DIM_FILE_CARDINALITY / NUM_PARTITION;
|
||||||
|
ParallelIndexSupervisorTask task = runTestTask(
|
||||||
|
new SingleDimensionPartitionsSpec(targetRowsPerSegment, null, DIM1, false),
|
||||||
|
false);
|
||||||
|
Map<String, Object> expectedReports = buildExpectedTaskReportParallel(
|
||||||
|
task.getId(),
|
||||||
|
ImmutableList.of(),
|
||||||
|
new RowIngestionMetersTotals(600, 0, 0, 0));
|
||||||
|
Map<String, Object> actualReports = runTaskAndGetReports(task, TaskState.SUCCESS);
|
||||||
|
compareTaskReports(expectedReports, actualReports);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAppendLinearlyPartitionedSegmentsToHashPartitionedDatasourceSuccessfullyAppend()
|
public void testAppendLinearlyPartitionedSegmentsToHashPartitionedDatasourceSuccessfullyAppend()
|
||||||
{
|
{
|
||||||
@ -250,32 +268,29 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
|
|||||||
final int targetRowsPerSegment = NUM_ROW / DIM_FILE_CARDINALITY / NUM_PARTITION;
|
final int targetRowsPerSegment = NUM_ROW / DIM_FILE_CARDINALITY / NUM_PARTITION;
|
||||||
final Set<DataSegment> publishedSegments = new HashSet<>();
|
final Set<DataSegment> publishedSegments = new HashSet<>();
|
||||||
publishedSegments.addAll(
|
publishedSegments.addAll(
|
||||||
runTestTask(
|
runTask(runTestTask(
|
||||||
new SingleDimensionPartitionsSpec(
|
new SingleDimensionPartitionsSpec(
|
||||||
targetRowsPerSegment,
|
targetRowsPerSegment,
|
||||||
null,
|
null,
|
||||||
DIM1,
|
DIM1,
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
TaskState.SUCCESS,
|
|
||||||
false
|
false
|
||||||
)
|
), TaskState.SUCCESS)
|
||||||
);
|
);
|
||||||
// Append
|
// Append
|
||||||
publishedSegments.addAll(
|
publishedSegments.addAll(
|
||||||
runTestTask(
|
runTask(runTestTask(
|
||||||
new DynamicPartitionsSpec(5, null),
|
new DynamicPartitionsSpec(5, null),
|
||||||
TaskState.SUCCESS,
|
|
||||||
true
|
true
|
||||||
)
|
), TaskState.SUCCESS)
|
||||||
);
|
);
|
||||||
// And append again
|
// And append again
|
||||||
publishedSegments.addAll(
|
publishedSegments.addAll(
|
||||||
runTestTask(
|
runTask(runTestTask(
|
||||||
new DynamicPartitionsSpec(10, null),
|
new DynamicPartitionsSpec(10, null),
|
||||||
TaskState.SUCCESS,
|
|
||||||
true
|
true
|
||||||
)
|
), TaskState.SUCCESS)
|
||||||
);
|
);
|
||||||
|
|
||||||
final Map<Interval, List<DataSegment>> intervalToSegments = new HashMap<>();
|
final Map<Interval, List<DataSegment>> intervalToSegments = new HashMap<>();
|
||||||
@ -306,14 +321,13 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private Set<DataSegment> runTestTask(
|
private ParallelIndexSupervisorTask runTestTask(
|
||||||
PartitionsSpec partitionsSpec,
|
PartitionsSpec partitionsSpec,
|
||||||
TaskState expectedTaskState,
|
|
||||||
boolean appendToExisting
|
boolean appendToExisting
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (isUseInputFormatApi()) {
|
if (isUseInputFormatApi()) {
|
||||||
return runTestTask(
|
return createTask(
|
||||||
TIMESTAMP_SPEC,
|
TIMESTAMP_SPEC,
|
||||||
DIMENSIONS_SPEC,
|
DIMENSIONS_SPEC,
|
||||||
INPUT_FORMAT,
|
INPUT_FORMAT,
|
||||||
@ -323,11 +337,10 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
|
|||||||
TEST_FILE_NAME_PREFIX + "*",
|
TEST_FILE_NAME_PREFIX + "*",
|
||||||
partitionsSpec,
|
partitionsSpec,
|
||||||
maxNumConcurrentSubTasks,
|
maxNumConcurrentSubTasks,
|
||||||
expectedTaskState,
|
|
||||||
appendToExisting
|
appendToExisting
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
return runTestTask(
|
return createTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
@ -337,7 +350,6 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
|
|||||||
TEST_FILE_NAME_PREFIX + "*",
|
TEST_FILE_NAME_PREFIX + "*",
|
||||||
partitionsSpec,
|
partitionsSpec,
|
||||||
maxNumConcurrentSubTasks,
|
maxNumConcurrentSubTasks,
|
||||||
expectedTaskState,
|
|
||||||
appendToExisting
|
appendToExisting
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -27,7 +27,6 @@ import org.apache.druid.data.input.impl.DimensionsSpec;
|
|||||||
import org.apache.druid.data.input.impl.JsonInputFormat;
|
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||||
import org.apache.druid.data.input.impl.LocalInputSource;
|
import org.apache.druid.data.input.impl.LocalInputSource;
|
||||||
import org.apache.druid.data.input.impl.StringInputRowParser;
|
import org.apache.druid.data.input.impl.StringInputRowParser;
|
||||||
import org.apache.druid.indexer.IngestionState;
|
|
||||||
import org.apache.druid.indexer.TaskState;
|
import org.apache.druid.indexer.TaskState;
|
||||||
import org.apache.druid.indexing.common.LockGranularity;
|
import org.apache.druid.indexing.common.LockGranularity;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
@ -71,14 +70,12 @@ import java.nio.charset.StandardCharsets;
|
|||||||
import java.nio.file.Files;
|
import java.nio.file.Files;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
@RunWith(Parameterized.class)
|
@RunWith(Parameterized.class)
|
||||||
public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSupervisorTaskTest
|
public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSupervisorTaskTest
|
||||||
@ -335,7 +332,7 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
|
|||||||
Collections.emptyList()
|
Collections.emptyList()
|
||||||
);
|
);
|
||||||
Map<String, Object> actualReports = task.doGetLiveReports("full");
|
Map<String, Object> actualReports = task.doGetLiveReports("full");
|
||||||
Map<String, Object> expectedReports = getExpectedTaskReportParallel(
|
Map<String, Object> expectedReports = buildExpectedTaskReportParallel(
|
||||||
task.getId(),
|
task.getId(),
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new ParseExceptionReport(
|
new ParseExceptionReport(
|
||||||
@ -360,68 +357,6 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
|
|||||||
compareTaskReports(expectedReports, actualReports);
|
compareTaskReports(expectedReports, actualReports);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void compareTaskReports(
|
|
||||||
Map<String, Object> expectedReports,
|
|
||||||
Map<String, Object> actualReports
|
|
||||||
)
|
|
||||||
{
|
|
||||||
expectedReports = (Map<String, Object>) expectedReports.get("ingestionStatsAndErrors");
|
|
||||||
actualReports = (Map<String, Object>) actualReports.get("ingestionStatsAndErrors");
|
|
||||||
|
|
||||||
Assert.assertEquals(expectedReports.get("taskId"), actualReports.get("taskId"));
|
|
||||||
Assert.assertEquals(expectedReports.get("type"), actualReports.get("type"));
|
|
||||||
|
|
||||||
Map<String, Object> expectedPayload = (Map<String, Object>) expectedReports.get("payload");
|
|
||||||
Map<String, Object> actualPayload = (Map<String, Object>) actualReports.get("payload");
|
|
||||||
Assert.assertEquals(expectedPayload.get("ingestionState"), actualPayload.get("ingestionState"));
|
|
||||||
Assert.assertEquals(expectedPayload.get("rowStats"), actualPayload.get("rowStats"));
|
|
||||||
Assert.assertEquals(expectedPayload.get("ingestionState"), actualPayload.get("ingestionState"));
|
|
||||||
|
|
||||||
List<ParseExceptionReport> expectedParseExceptionReports = (List<ParseExceptionReport>) ((Map<String, Object>) expectedPayload.get("unparseableEvents"))
|
|
||||||
.get("buildSegments");
|
|
||||||
|
|
||||||
List<ParseExceptionReport> actualParseExceptionReports = (List<ParseExceptionReport>) ((Map<String, Object>) actualPayload.get("unparseableEvents"))
|
|
||||||
.get("buildSegments");
|
|
||||||
|
|
||||||
List<String> expectedMessages = expectedParseExceptionReports.stream().map((r) -> {
|
|
||||||
return r.getDetails().get(0);
|
|
||||||
}).collect(Collectors.toList());
|
|
||||||
List<String> actualMessages = actualParseExceptionReports.stream().map((r) -> {
|
|
||||||
return r.getDetails().get(0);
|
|
||||||
}).collect(Collectors.toList());
|
|
||||||
Assert.assertEquals(expectedMessages, actualMessages);
|
|
||||||
|
|
||||||
List<String> expectedInputs = expectedParseExceptionReports.stream().map((r) -> {
|
|
||||||
return r.getInput();
|
|
||||||
}).collect(Collectors.toList());
|
|
||||||
List<String> actualInputs = actualParseExceptionReports.stream().map((r) -> {
|
|
||||||
return r.getInput();
|
|
||||||
}).collect(Collectors.toList());
|
|
||||||
Assert.assertEquals(expectedInputs, actualInputs);
|
|
||||||
}
|
|
||||||
|
|
||||||
private Map<String, Object> getExpectedTaskReportParallel(
|
|
||||||
String taskId,
|
|
||||||
List<ParseExceptionReport> expectedUnparseableEvents,
|
|
||||||
RowIngestionMetersTotals expectedTotals
|
|
||||||
)
|
|
||||||
{
|
|
||||||
Map<String, Object> returnMap = new HashMap<>();
|
|
||||||
Map<String, Object> ingestionStatsAndErrors = new HashMap<>();
|
|
||||||
Map<String, Object> payload = new HashMap<>();
|
|
||||||
|
|
||||||
payload.put("ingestionState", IngestionState.COMPLETED);
|
|
||||||
payload.put("unparseableEvents", ImmutableMap.of("buildSegments", expectedUnparseableEvents));
|
|
||||||
payload.put("rowStats", ImmutableMap.of("totals", ImmutableMap.of("buildSegments", expectedTotals)));
|
|
||||||
|
|
||||||
ingestionStatsAndErrors.put("taskId", taskId);
|
|
||||||
ingestionStatsAndErrors.put("payload", payload);
|
|
||||||
ingestionStatsAndErrors.put("type", "ingestionStatsAndErrors");
|
|
||||||
|
|
||||||
returnMap.put("ingestionStatsAndErrors", ingestionStatsAndErrors);
|
|
||||||
return returnMap;
|
|
||||||
}
|
|
||||||
|
|
||||||
//
|
//
|
||||||
// Ingest all data.
|
// Ingest all data.
|
||||||
|
|
||||||
@ -475,15 +410,16 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
|
|||||||
|
|
||||||
Map<String, Object> expectedReports;
|
Map<String, Object> expectedReports;
|
||||||
if (useInputFormatApi) {
|
if (useInputFormatApi) {
|
||||||
expectedReports = getExpectedTaskReportSequential(
|
expectedReports = buildExpectedTaskReportSequential(
|
||||||
task.getId(),
|
task.getId(),
|
||||||
expectedUnparseableEvents,
|
expectedUnparseableEvents,
|
||||||
|
new RowIngestionMetersTotals(0, 0, 0, 0),
|
||||||
expectedTotals
|
expectedTotals
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
// when useInputFormatApi is false, maxConcurrentSubTasks=2 and it uses the single phase runner
|
// when useInputFormatApi is false, maxConcurrentSubTasks=2 and it uses the single phase runner
|
||||||
// instead of sequential runner
|
// instead of sequential runner
|
||||||
expectedReports = getExpectedTaskReportParallel(
|
expectedReports = buildExpectedTaskReportParallel(
|
||||||
task.getId(),
|
task.getId(),
|
||||||
expectedUnparseableEvents,
|
expectedUnparseableEvents,
|
||||||
expectedTotals
|
expectedTotals
|
||||||
@ -494,65 +430,6 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
|
|||||||
System.out.println(actualReports);
|
System.out.println(actualReports);
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<String, Object> getExpectedTaskReportSequential(
|
|
||||||
String taskId,
|
|
||||||
List<ParseExceptionReport> expectedUnparseableEvents,
|
|
||||||
RowIngestionMetersTotals expectedTotals
|
|
||||||
)
|
|
||||||
{
|
|
||||||
Map<String, Object> returnMap = new HashMap<>();
|
|
||||||
Map<String, Object> ingestionStatsAndErrors = new HashMap<>();
|
|
||||||
Map<String, Object> payload = new HashMap<>();
|
|
||||||
|
|
||||||
payload.put("ingestionState", IngestionState.COMPLETED);
|
|
||||||
payload.put(
|
|
||||||
"unparseableEvents",
|
|
||||||
ImmutableMap.of(
|
|
||||||
"determinePartitions", ImmutableList.of(),
|
|
||||||
"buildSegments", expectedUnparseableEvents
|
|
||||||
)
|
|
||||||
);
|
|
||||||
Map<String, Object> emptyAverageMinuteMap = ImmutableMap.of(
|
|
||||||
"processed", 0.0,
|
|
||||||
"unparseable", 0.0,
|
|
||||||
"thrownAway", 0.0,
|
|
||||||
"processedWithError", 0.0
|
|
||||||
);
|
|
||||||
|
|
||||||
Map<String, Object> emptyAverages = ImmutableMap.of(
|
|
||||||
"1m", emptyAverageMinuteMap,
|
|
||||||
"5m", emptyAverageMinuteMap,
|
|
||||||
"15m", emptyAverageMinuteMap
|
|
||||||
);
|
|
||||||
|
|
||||||
payload.put(
|
|
||||||
"rowStats",
|
|
||||||
ImmutableMap.of(
|
|
||||||
"movingAverages",
|
|
||||||
ImmutableMap.of(
|
|
||||||
"determinePartitions",
|
|
||||||
emptyAverages,
|
|
||||||
"buildSegments",
|
|
||||||
emptyAverages
|
|
||||||
),
|
|
||||||
"totals",
|
|
||||||
ImmutableMap.of(
|
|
||||||
"determinePartitions",
|
|
||||||
new RowIngestionMetersTotals(0, 0, 0, 0),
|
|
||||||
"buildSegments",
|
|
||||||
expectedTotals
|
|
||||||
)
|
|
||||||
)
|
|
||||||
);
|
|
||||||
|
|
||||||
ingestionStatsAndErrors.put("taskId", taskId);
|
|
||||||
ingestionStatsAndErrors.put("payload", payload);
|
|
||||||
ingestionStatsAndErrors.put("type", "ingestionStatsAndErrors");
|
|
||||||
|
|
||||||
returnMap.put("ingestionStatsAndErrors", ingestionStatsAndErrors);
|
|
||||||
return returnMap;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testPublishEmptySegments()
|
public void testPublishEmptySegments()
|
||||||
{
|
{
|
||||||
|
@ -0,0 +1,106 @@
|
|||||||
|
/*
|
||||||
|
* 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.incremental;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class MutableRowIngestionMeters implements RowIngestionMeters
|
||||||
|
{
|
||||||
|
private long processed;
|
||||||
|
private long processedWithError;
|
||||||
|
private long unparseable;
|
||||||
|
private long thrownAway;
|
||||||
|
|
||||||
|
public MutableRowIngestionMeters()
|
||||||
|
{
|
||||||
|
this.processed = 0;
|
||||||
|
this.processedWithError = 0;
|
||||||
|
this.unparseable = 0;
|
||||||
|
this.thrownAway = 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getProcessed()
|
||||||
|
{
|
||||||
|
return processed;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void incrementProcessed()
|
||||||
|
{
|
||||||
|
processed++;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getProcessedWithError()
|
||||||
|
{
|
||||||
|
return processedWithError;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void incrementProcessedWithError()
|
||||||
|
{
|
||||||
|
processedWithError++;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getUnparseable()
|
||||||
|
{
|
||||||
|
return unparseable;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void incrementUnparseable()
|
||||||
|
{
|
||||||
|
unparseable++;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getThrownAway()
|
||||||
|
{
|
||||||
|
return thrownAway;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void incrementThrownAway()
|
||||||
|
{
|
||||||
|
thrownAway++;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RowIngestionMetersTotals getTotals()
|
||||||
|
{
|
||||||
|
return new RowIngestionMetersTotals(processed, processedWithError, thrownAway, unparseable);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, Object> getMovingAverages()
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
public void addRowIngestionMetersTotals(RowIngestionMetersTotals rowIngestionMetersTotals)
|
||||||
|
{
|
||||||
|
this.processed += rowIngestionMetersTotals.getProcessed();
|
||||||
|
this.processedWithError += rowIngestionMetersTotals.getProcessedWithError();
|
||||||
|
this.unparseable += rowIngestionMetersTotals.getUnparseable();
|
||||||
|
this.thrownAway += rowIngestionMetersTotals.getThrownAway();
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,46 @@
|
|||||||
|
/*
|
||||||
|
* 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.incremental;
|
||||||
|
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
public class MutableRowIngestionMetersTest
|
||||||
|
{
|
||||||
|
@Test
|
||||||
|
public void testIncrement()
|
||||||
|
{
|
||||||
|
MutableRowIngestionMeters mutableRowIngestionMeters = new MutableRowIngestionMeters();
|
||||||
|
mutableRowIngestionMeters.incrementProcessed();
|
||||||
|
mutableRowIngestionMeters.incrementProcessedWithError();
|
||||||
|
mutableRowIngestionMeters.incrementUnparseable();
|
||||||
|
mutableRowIngestionMeters.incrementThrownAway();
|
||||||
|
Assert.assertEquals(mutableRowIngestionMeters.getTotals(), new RowIngestionMetersTotals(1, 1, 1, 1));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAddRowIngestionMetersTotals()
|
||||||
|
{
|
||||||
|
MutableRowIngestionMeters mutableRowIngestionMeters = new MutableRowIngestionMeters();
|
||||||
|
RowIngestionMetersTotals rowIngestionMetersTotals = new RowIngestionMetersTotals(10, 1, 0, 1);
|
||||||
|
mutableRowIngestionMeters.addRowIngestionMetersTotals(rowIngestionMetersTotals);
|
||||||
|
Assert.assertEquals(mutableRowIngestionMeters.getTotals(), rowIngestionMetersTotals);
|
||||||
|
}
|
||||||
|
}
|
Loading…
x
Reference in New Issue
Block a user