From d9563f27e747b21d3a5aef593791c18864ededc4 Mon Sep 17 00:00:00 2001 From: Amatya Date: Mon, 12 Jun 2023 16:36:26 +0530 Subject: [PATCH 01/43] Add support for concurrent batch Append and Replace --- .../sqlserver/SQLServerConnectorTest.java | 16 +- .../storage/mysql/MySQLConnectorTest.java | 2 +- .../postgresql/PostgreSQLConnectorTest.java | 20 +- .../MetadataStorageUpdaterJobSpec.java | 1 + .../SegmentTransactionalAppendAction.java | 239 +++ .../SegmentTransactionalReplaceAction.java | 264 ++++ .../indexing/common/actions/TaskAction.java | 2 + .../indexing/common/actions/TaskLocks.java | 29 +- .../common/task/AbstractBatchIndexTask.java | 24 +- .../indexing/common/task/AppendTask.java | 336 ++++ .../AppenderatorDriverRealtimeIndexTask.java | 2 +- .../druid/indexing/common/task/IndexTask.java | 23 +- .../indexing/common/task/ReplaceTask.java | 256 ++++ .../druid/indexing/common/task/Task.java | 4 +- .../indexing/common/task/TaskLockHelper.java | 13 +- .../druid/indexing/common/task/Tasks.java | 2 +- .../parallel/ParallelIndexSupervisorTask.java | 24 +- .../SinglePhaseParallelIndexTaskRunner.java | 8 +- .../overlord/LockRequestForNewSegment.java | 16 +- .../druid/indexing/overlord/TaskLockbox.java | 25 +- .../overlord/TimeChunkLockRequest.java | 3 + .../SeekableStreamIndexTask.java | 2 +- ...penderatorDriverRealtimeIndexTaskTest.java | 2 +- .../task/ConcurrentReplaceAndAppendTest.java | 1361 +++++++++++++++++ .../common/task/IngestionTestBase.java | 31 +- .../SinglePhaseParallelIndexingTest.java | 5 +- ...TestIndexerMetadataStorageCoordinator.java | 17 +- .../metadata/MetadataStorageConnector.java | 2 + .../metadata/MetadataStorageTablesConfig.java | 14 +- .../IndexerMetadataStorageCoordinator.java | 14 + .../druid/indexing/overlord/TaskLockInfo.java | 66 + .../IndexerSQLMetadataStorageCoordinator.java | 463 +++++- .../druid/metadata/SQLMetadataConnector.java | 34 + .../SqlSegmentsMetadataManagerProvider.java | 1 + .../TransactionalSegmentPublisher.java | 5 + .../org/apache/druid/cli/CreateTables.java | 1 + 36 files changed, 3203 insertions(+), 124 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppendTask.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/ReplaceTask.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java create mode 100644 server/src/main/java/org/apache/druid/indexing/overlord/TaskLockInfo.java diff --git a/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java b/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java index a31159e88134..ab3d9c37fe23 100644 --- a/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java +++ b/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java @@ -37,19 +37,7 @@ public void testIsTransientException() SQLServerConnector connector = new SQLServerConnector( Suppliers.ofInstance(new MetadataStorageConnectorConfig()), Suppliers.ofInstance( - new MetadataStorageTablesConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + MetadataStorageTablesConfig.fromBase(null) ) ); @@ -70,7 +58,7 @@ public void testLimitClause() SQLServerConnector connector = new SQLServerConnector( Suppliers.ofInstance(new MetadataStorageConnectorConfig()), Suppliers.ofInstance( - new MetadataStorageTablesConfig(null, null, null, null, null, null, null, null, null, null, null) + MetadataStorageTablesConfig.fromBase(null) ) ); Assert.assertEquals("FETCH NEXT 100 ROWS ONLY", connector.limitClause(100)); diff --git a/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorTest.java b/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorTest.java index 23ce46282232..2498e8a65c17 100644 --- a/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorTest.java +++ b/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorTest.java @@ -44,7 +44,7 @@ public String getDriverClassName() private static final Supplier CONNECTOR_CONFIG_SUPPLIER = MetadataStorageConnectorConfig::new; private static final Supplier TABLES_CONFIG_SUPPLIER = - () -> new MetadataStorageTablesConfig(null, null, null, null, null, null, null, null, null, null, null); + () -> MetadataStorageTablesConfig.fromBase(null); @Test diff --git a/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java b/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java index 08f3c333a1fb..3b6c9aace521 100644 --- a/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java +++ b/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java @@ -35,21 +35,7 @@ public void testIsTransientException() { PostgreSQLConnector connector = new PostgreSQLConnector( Suppliers.ofInstance(new MetadataStorageConnectorConfig()), - Suppliers.ofInstance( - new MetadataStorageTablesConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) - ), + Suppliers.ofInstance(MetadataStorageTablesConfig.fromBase(null)), new PostgreSQLConnectorConfig(), new PostgreSQLTablesConfig() ); @@ -68,9 +54,7 @@ public void testLimitClause() { PostgreSQLConnector connector = new PostgreSQLConnector( Suppliers.ofInstance(new MetadataStorageConnectorConfig()), - Suppliers.ofInstance( - new MetadataStorageTablesConfig(null, null, null, null, null, null, null, null, null, null, null) - ), + Suppliers.ofInstance(MetadataStorageTablesConfig.fromBase(null)), new PostgreSQLConnectorConfig(), new PostgreSQLTablesConfig() ); diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java index 0f8ca0acfccc..b262e9b4def6 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java @@ -90,6 +90,7 @@ public MetadataStorageTablesConfig getMetadataStorageTablesConfig() null, null, null, + null, segmentTable, null, null, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java new file mode 100644 index 000000000000..bc202a1a0fe1 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java @@ -0,0 +1,239 @@ +/* + * 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.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.task.IndexTaskUtils; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.CriticalAction; +import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.indexing.overlord.TaskLockInfo; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.SegmentUtils; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Append segments to metadata storage. The segment versions must all be less than or equal to a lock held by + * your task for the segment intervals. + */ +public class SegmentTransactionalAppendAction implements TaskAction +{ + private final Set segments; + + @Nullable + private final DataSourceMetadata startMetadata; + @Nullable + private final DataSourceMetadata endMetadata; + @Nullable + private final String dataSource; + + public static SegmentTransactionalAppendAction appendAction( + Set segments, + @Nullable DataSourceMetadata startMetadata, + @Nullable DataSourceMetadata endMetadata + ) + { + return new SegmentTransactionalAppendAction(segments, startMetadata, endMetadata, null); + } + + @JsonCreator + private SegmentTransactionalAppendAction( + @JsonProperty("segments") @Nullable Set segments, + @JsonProperty("startMetadata") @Nullable DataSourceMetadata startMetadata, + @JsonProperty("endMetadata") @Nullable DataSourceMetadata endMetadata, + @JsonProperty("dataSource") @Nullable String dataSource + ) + { + this.segments = segments == null ? ImmutableSet.of() : ImmutableSet.copyOf(segments); + this.startMetadata = startMetadata; + this.endMetadata = endMetadata; + this.dataSource = dataSource; + } + + @JsonProperty + public Set getSegments() + { + return segments; + } + + @JsonProperty + @Nullable + public DataSourceMetadata getStartMetadata() + { + return startMetadata; + } + + @JsonProperty + @Nullable + public DataSourceMetadata getEndMetadata() + { + return endMetadata; + } + + @JsonProperty + @Nullable + public String getDataSource() + { + return dataSource; + } + + @Override + public TypeReference getReturnTypeReference() + { + return new TypeReference() + { + }; + } + + /** + * Performs some sanity checks and publishes the given segments. + */ + @Override + public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) + { + final SegmentPublishResult retVal; + + if (segments.isEmpty()) { + // A stream ingestion task didn't ingest any rows and created no segments (e.g., all records were unparseable), + // but still needs to update metadata with the progress that the task made. + try { + retVal = toolbox.getIndexerMetadataStorageCoordinator().commitMetadataOnly( + dataSource, + startMetadata, + endMetadata + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + return retVal; + } + + final Set allSegments = new HashSet<>(segments); + + String datasource = task.getDataSource(); + Map replaceLocks = new HashMap<>(); + for (TaskLock lock : TaskLocks.findReplaceLocksForSegments(datasource, toolbox.getTaskLockbox(), segments)) { + replaceLocks.put(lock.getInterval(), lock); + } + Map appendSegmentLockMap = new HashMap<>(); + Set taskLockInfos = new HashSet<>(); + for (TaskLock taskLock : replaceLocks.values()) { + taskLockInfos.add(getTaskLockInfo(taskLock)); + } + + for (DataSegment segment : segments) { + Interval interval = segment.getInterval(); + for (Interval key : replaceLocks.keySet()) { + if (key.contains(interval)) { + appendSegmentLockMap.put(segment, getTaskLockInfo(replaceLocks.get(key))); + } + } + } + + try { + retVal = toolbox.getTaskLockbox().doInCriticalSection( + task, + allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toList()), + CriticalAction.builder() + .onValidLocks( + () -> toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments( + segments, + null, + startMetadata, + endMetadata, + appendSegmentLockMap, + taskLockInfos, + true + ) + ) + .onInvalidLocks( + () -> SegmentPublishResult.fail( + "Invalid task locks. Maybe they are revoked by a higher priority task." + + " Please check the overlord log for details." + ) + ) + .build() + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + + // Emit metrics + final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); + IndexTaskUtils.setTaskDimensions(metricBuilder, task); + + if (retVal.isSuccess()) { + toolbox.getEmitter().emit(metricBuilder.build("segment/txn/success", 1)); + } else { + toolbox.getEmitter().emit(metricBuilder.build("segment/txn/failure", 1)); + } + + // getSegments() should return an empty set if announceHistoricalSegments() failed + for (DataSegment segment : retVal.getSegments()) { + metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); + metricBuilder.setDimension( + DruidMetrics.PARTITIONING_TYPE, + segment.getShardSpec() == null ? null : segment.getShardSpec().getType() + ); + toolbox.getEmitter().emit(metricBuilder.build("segment/added/bytes", segment.getSize())); + } + + return retVal; + } + + + private TaskLockInfo getTaskLockInfo(TaskLock taskLock) + { + return new TaskLockInfo(taskLock.getInterval(), taskLock.getVersion()); + } + + @Override + public boolean isAudited() + { + return true; + } + + @Override + public String toString() + { + return "SegmentTransactionalInsertAction{" + + "segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + + ", startMetadata=" + startMetadata + + ", endMetadata=" + endMetadata + + ", dataSource='" + dataSource + '\'' + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java new file mode 100644 index 000000000000..23c8fae0aa2e --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -0,0 +1,264 @@ +/* + * 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.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.task.IndexTaskUtils; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.CriticalAction; +import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.indexing.overlord.TaskLockInfo; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.SegmentUtils; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Replace segments in metadata storage. The segment versions must all be less than or equal to a lock held by + * your task for the segment intervals. + */ +public class SegmentTransactionalReplaceAction implements TaskAction +{ + /** + * Set of segments that was fully overshadowed by new segments, {@link SegmentTransactionalReplaceAction#segments} + */ + @Nullable + private final Set segmentsToBeOverwritten; + /** + * Set of segments to be inserted into metadata storage + */ + private final Set segments; + /** + * Set of segments to be dropped (mark unused) when new segments, {@link SegmentTransactionalReplaceAction#segments}, + * are inserted into metadata storage. + */ + @Nullable + private final Set segmentsToBeDropped; + + @Nullable + private final DataSourceMetadata startMetadata; + @Nullable + private final DataSourceMetadata endMetadata; + @Nullable + private final String dataSource; + + public static SegmentTransactionalReplaceAction overwriteAction( + @Nullable Set segmentsToBeOverwritten, + @Nullable Set segmentsToBeDropped, + Set segmentsToPublish + ) + { + return new SegmentTransactionalReplaceAction(segmentsToBeOverwritten, segmentsToBeDropped, segmentsToPublish, null, null, null); + } + + @JsonCreator + private SegmentTransactionalReplaceAction( + @JsonProperty("segmentsToBeOverwritten") @Nullable Set segmentsToBeOverwritten, + @JsonProperty("segmentsToBeDropped") @Nullable Set segmentsToBeDropped, + @JsonProperty("segments") @Nullable Set segments, + @JsonProperty("startMetadata") @Nullable DataSourceMetadata startMetadata, + @JsonProperty("endMetadata") @Nullable DataSourceMetadata endMetadata, + @JsonProperty("dataSource") @Nullable String dataSource + ) + { + this.segmentsToBeOverwritten = segmentsToBeOverwritten; + this.segmentsToBeDropped = segmentsToBeDropped; + this.segments = segments == null ? ImmutableSet.of() : ImmutableSet.copyOf(segments); + this.startMetadata = startMetadata; + this.endMetadata = endMetadata; + this.dataSource = dataSource; + } + + @JsonProperty + @Nullable + public Set getSegmentsToBeOverwritten() + { + return segmentsToBeOverwritten; + } + + @JsonProperty + @Nullable + public Set getSegmentsToBeDropped() + { + return segmentsToBeDropped; + } + + @JsonProperty + public Set getSegments() + { + return segments; + } + + @JsonProperty + @Nullable + public DataSourceMetadata getStartMetadata() + { + return startMetadata; + } + + @JsonProperty + @Nullable + public DataSourceMetadata getEndMetadata() + { + return endMetadata; + } + + @JsonProperty + @Nullable + public String getDataSource() + { + return dataSource; + } + + @Override + public TypeReference getReturnTypeReference() + { + return new TypeReference() + { + }; + } + + /** + * Performs some sanity checks and publishes the given segments. + */ + @Override + public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) + { + final SegmentPublishResult retVal; + + if (segments.isEmpty()) { + // A stream ingestion task didn't ingest any rows and created no segments (e.g., all records were unparseable), + // but still needs to update metadata with the progress that the task made. + try { + retVal = toolbox.getIndexerMetadataStorageCoordinator().commitMetadataOnly( + dataSource, + startMetadata, + endMetadata + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + return retVal; + } + + final Set allSegments = new HashSet<>(segments); + + TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), allSegments); + + String datasource = task.getDataSource(); + Map replaceLocks = new HashMap<>(); + for (TaskLock lock : TaskLocks.findReplaceLocksForSegments(datasource, toolbox.getTaskLockbox(), segments)) { + replaceLocks.put(lock.getInterval(), lock); + } + Set taskLockInfos = new HashSet<>(); + for (TaskLock taskLock : replaceLocks.values()) { + taskLockInfos.add(getTaskLockInfo(taskLock)); + } + + try { + retVal = toolbox.getTaskLockbox().doInCriticalSection( + task, + allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toList()), + CriticalAction.builder() + .onValidLocks( + () -> toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments( + segments, + segmentsToBeDropped, + startMetadata, + endMetadata, + null, + taskLockInfos, + false + ) + ) + .onInvalidLocks( + () -> SegmentPublishResult.fail( + "Invalid task locks. Maybe they are revoked by a higher priority task." + + " Please check the overlord log for details." + ) + ) + .build() + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + + // Emit metrics + final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); + IndexTaskUtils.setTaskDimensions(metricBuilder, task); + + if (retVal.isSuccess()) { + toolbox.getEmitter().emit(metricBuilder.build("segment/txn/success", 1)); + } else { + toolbox.getEmitter().emit(metricBuilder.build("segment/txn/failure", 1)); + } + + // getSegments() should return an empty set if announceHistoricalSegments() failed + for (DataSegment segment : retVal.getSegments()) { + metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); + metricBuilder.setDimension( + DruidMetrics.PARTITIONING_TYPE, + segment.getShardSpec() == null ? null : segment.getShardSpec().getType() + ); + toolbox.getEmitter().emit(metricBuilder.build("segment/added/bytes", segment.getSize())); + } + + return retVal; + } + + private TaskLockInfo getTaskLockInfo(TaskLock taskLock) + { + return new TaskLockInfo(taskLock.getInterval(), taskLock.getVersion()); + } + + @Override + public boolean isAudited() + { + return true; + } + + @Override + public String toString() + { + return "SegmentTransactionalInsertAction{" + + "segmentsToBeOverwritten=" + SegmentUtils.commaSeparatedIdentifiers(segmentsToBeOverwritten) + + ", segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + + ", startMetadata=" + startMetadata + + ", endMetadata=" + endMetadata + + ", dataSource='" + dataSource + '\'' + + ", segmentsToBeDropped=" + SegmentUtils.commaSeparatedIdentifiers(segmentsToBeDropped) + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java index 18e373727790..171d53b9cdd6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java @@ -36,6 +36,8 @@ @JsonSubTypes.Type(name = "lockRelease", value = LockReleaseAction.class), @JsonSubTypes.Type(name = "segmentInsertion", value = SegmentInsertAction.class), @JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class), + @JsonSubTypes.Type(name = "segmentTransactionalAppend", value = SegmentTransactionalAppendAction.class), + @JsonSubTypes.Type(name = "segmentTransactionalReplace", value = SegmentTransactionalReplaceAction.class), // Type name doesn't correspond to the name of the class for backward compatibility. @JsonSubTypes.Type(name = "segmentListUsed", value = RetrieveUsedSegmentsAction.class), // Type name doesn't correspond to the name of the class for backward compatibility. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java index a7feb4fc5a2a..ab4d0a981a16 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java @@ -23,6 +23,7 @@ import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.SegmentLock; import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TimeChunkLock; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.TaskLockbox; @@ -33,9 +34,11 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map.Entry; import java.util.NavigableMap; +import java.util.Set; import java.util.TreeMap; public class TaskLocks @@ -96,7 +99,8 @@ public static boolean isLockCoversSegments( final TimeChunkLock timeChunkLock = (TimeChunkLock) lock; return timeChunkLock.getInterval().contains(segment.getInterval()) && timeChunkLock.getDataSource().equals(segment.getDataSource()) - && timeChunkLock.getVersion().compareTo(segment.getVersion()) >= 0; + && (timeChunkLock.getVersion().compareTo(segment.getVersion()) >= 0 + || TaskLockType.APPEND.equals(timeChunkLock.getType())); } else { final SegmentLock segmentLock = (SegmentLock) lock; return segmentLock.getInterval().contains(segment.getInterval()) @@ -110,6 +114,29 @@ public static boolean isLockCoversSegments( ); } + public static Set findReplaceLocksForSegments( + final String datasource, + final TaskLockbox taskLockbox, + final Collection segments + ) + { + final Set found = new HashSet<>(); + final Set locks = taskLockbox.getAllReplaceLocksForDatasource(datasource); + segments.forEach(segment -> { + + locks.forEach(lock -> { + if (lock.getGranularity() == LockGranularity.TIME_CHUNK) { + final TimeChunkLock timeChunkLock = (TimeChunkLock) lock; + if (timeChunkLock.getInterval().contains(segment.getInterval()) + && timeChunkLock.getDataSource().equals(segment.getDataSource())) { + found.add(lock); + } + } + }); + }); + return found; + } + public static List findLocksForSegments( final Task task, final TaskLockbox taskLockbox, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index a4cd183c01bb..1d0129edf4a9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -297,15 +297,14 @@ public boolean determineLockGranularityAndTryLock(TaskActionClient client, List< Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK ); IngestionMode ingestionMode = getIngestionMode(); - final boolean useSharedLock = ingestionMode == IngestionMode.APPEND - && getContextValue(Tasks.USE_SHARED_LOCK, false); + final TaskLockType taskLockType = TaskLockType.valueOf(getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name())); // Respect task context value most. if (forceTimeChunkLock || ingestionMode == IngestionMode.REPLACE) { log.info( "forceTimeChunkLock[%s] is set to true or mode[%s] is replace. Use timeChunk lock", forceTimeChunkLock, ingestionMode ); - taskLockHelper = new TaskLockHelper(false, useSharedLock); + taskLockHelper = new TaskLockHelper(false, taskLockType); if (!intervals.isEmpty()) { return tryTimeChunkLock(client, intervals); } else { @@ -314,7 +313,7 @@ public boolean determineLockGranularityAndTryLock(TaskActionClient client, List< } else { if (!intervals.isEmpty()) { final LockGranularityDetermineResult result = determineSegmentGranularity(client, intervals); - taskLockHelper = new TaskLockHelper(result.lockGranularity == LockGranularity.SEGMENT, useSharedLock); + taskLockHelper = new TaskLockHelper(result.lockGranularity == LockGranularity.SEGMENT, taskLockType); return tryLockWithDetermineResult(client, result); } else { // This branch is the only one that will not initialize taskLockHelper. @@ -342,11 +341,11 @@ boolean determineLockGranularityAndTryLockWithSegments( Tasks.FORCE_TIME_CHUNK_LOCK_KEY, Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK ); - final boolean useSharedLock = getContextValue(Tasks.USE_SHARED_LOCK, false); + final TaskLockType taskLockType = TaskLockType.valueOf(getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name())); if (forceTimeChunkLock) { log.info("[%s] is set to true in task context. Use timeChunk lock", Tasks.FORCE_TIME_CHUNK_LOCK_KEY); - taskLockHelper = new TaskLockHelper(false, useSharedLock); + taskLockHelper = new TaskLockHelper(false, taskLockType); segmentCheckFunction.accept(LockGranularity.TIME_CHUNK, segments); return tryTimeChunkLock( client, @@ -354,7 +353,7 @@ boolean determineLockGranularityAndTryLockWithSegments( ); } else { final LockGranularityDetermineResult result = determineSegmentGranularity(segments); - taskLockHelper = new TaskLockHelper(result.lockGranularity == LockGranularity.SEGMENT, useSharedLock); + taskLockHelper = new TaskLockHelper(result.lockGranularity == LockGranularity.SEGMENT, taskLockType); segmentCheckFunction.accept(result.lockGranularity, segments); return tryLockWithDetermineResult(client, result); } @@ -430,7 +429,8 @@ protected boolean tryTimeChunkLock(TaskActionClient client, List inter } prev = cur; - final TaskLock lock = client.submit(new TimeChunkLockTryAcquireAction(TaskLockType.EXCLUSIVE, cur)); + final TaskLockType taskLockType = TaskLockType.valueOf(getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name())); + final TaskLock lock = client.submit(new TimeChunkLockTryAcquireAction(taskLockType, cur)); if (lock == null) { return false; } @@ -670,7 +670,8 @@ public static String findVersion(Map versions, Interval interv public static NonnullPair findIntervalAndVersion( TaskToolbox toolbox, IngestionSpec ingestionSpec, - DateTime timestamp + DateTime timestamp, + TaskLockType taskLockType ) throws IOException { // This method is called whenever subtasks need to allocate a new segment via the supervisor task. @@ -724,9 +725,12 @@ public static NonnullPair findIntervalAndVersion( } } // We don't have a lock for this interval, so we should lock it now. + if (taskLockType == null) { + taskLockType = TaskLockType.EXCLUSIVE; + } final TaskLock lock = Preconditions.checkNotNull( toolbox.getTaskActionClient().submit( - new TimeChunkLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval) + new TimeChunkLockTryAcquireAction(taskLockType, interval) ), "Cannot acquire a lock for interval[%s]", interval diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppendTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppendTask.java new file mode 100644 index 000000000000..bbcaa8ece272 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppendTask.java @@ -0,0 +1,336 @@ +/* + * 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.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.indexer.TaskStatus; +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.TaskToolbox; +import org.apache.druid.indexing.common.actions.SegmentAllocateAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; +import org.apache.druid.indexing.common.actions.SurrogateTaskActionClient; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; +import org.apache.druid.indexing.common.config.TaskConfig; +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.granularity.DurationGranularity; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; +import org.apache.druid.server.security.ResourceAction; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.NumberedPartialShardSpec; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nonnull; +import java.io.IOException; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; + +/** + * A test Task which mimics an appending task by having similar interactions with the overlord. + * + * Begins running by acquiring an APPEND lock and immediately allocates pending segments. + * + * Task ends after publishing these pending segments and relevant metadata entries in a transaction + * Replace lock exists with version V3 + * V0 -> PS0 -> append task begins + * V1 -> (S1-0) -> replace task begins and published and completed + * V2 -> (S2-0) -> replace task begins and published and completed + * V3 -> Replace acquired lock + * + * append task publishes PS0 -> S0-0, S1-1, S2-1 ; (S0-0, V3); Append task has completed + * + * + * (S3-0, 1) today + * (S3-0, 2), (S3-1, 2) needs to happen + * V3 replace task finishes -> (S3-0, S0-0 == S3-1) + * + * segment metadata -> Publish all pending segments and also create copies for greater versions for which used segments exist + * forward metadata -> Publish mapping of the original segments to the EXCLUSIVE lock held for the same interval when present + */ +public class AppendTask extends AbstractTask +{ + private final Interval interval; + private final Granularity segmentGranularity; + private final String lockType; + private final int priority; + private final int numPartitions; + private final CountDownLatch readyLatch = new CountDownLatch(1); + private final CountDownLatch runLatch = new CountDownLatch(1); + private final CountDownLatch segmentAllocationComplete = new CountDownLatch(1); + private final CountDownLatch runComplete = new CountDownLatch(1); + private final CountDownLatch readyComplete = new CountDownLatch(1); + + private final Set pendingSegments = new HashSet<>(); + + private TaskToolbox toolbox; + private final AtomicInteger sequenceId = new AtomicInteger(0); + + public AppendTask( + @JsonProperty("id") String id, + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval, + @JsonProperty("segmentGranularity") Granularity segmentGranularity, + @JsonProperty("context") Map context + ) + { + super( + id == null ? StringUtils.format("replace_%s_%s", DateTimes.nowUtc(), UUID.randomUUID().toString()) : id, + dataSource == null ? "none" : dataSource, + context, + IngestionMode.APPEND + ); + this.interval = interval; + this.segmentGranularity = segmentGranularity; + this.lockType = getContextValue(Tasks.TASK_LOCK_TYPE, "EXCLUSIVE"); + this.priority = getContextValue(Tasks.PRIORITY_KEY, 0); + this.numPartitions = getContextValue("numPartitions", 0); + } + + @Override + public String getType() + { + return "replace"; + } + + @Nonnull + @JsonIgnore + @Override + public Set getInputSourceResources() + { + return ImmutableSet.of(); + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + readyLatch.await(); + return tryTimeChunkLockSingleInterval( + new SurrogateTaskActionClient(getId(), taskActionClient), + interval, + TaskLockType.valueOf(lockType) + ); + } + + private boolean tryTimeChunkLockSingleInterval(TaskActionClient client, Interval interval, TaskLockType lockType) + throws IOException + { + final TaskLock lock = client.submit(new TimeChunkLockTryAcquireAction(lockType, interval)); + if (lock == null) { + return false; + } + if (lock.isRevoked()) { + throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", interval)); + } + return true; + } + + @Override + public void stopGracefully(TaskConfig taskConfig) + { + } + + @Override + public TaskStatus runTask(TaskToolbox toolbox) throws Exception + { + this.toolbox = toolbox; + readyComplete.countDown(); + + //final Set pendingSegments = allocatePendingSegments(toolbox); + + segmentAllocationComplete.await(); + + runLatch.await(); + + if (publishSegments(toolbox, convertPendingSegments(pendingSegments))) { + return TaskStatus.success(getId()); + } + return TaskStatus.failure(getId(), "Failed to append segments"); + } + + @Override + public void cleanUp(TaskToolbox toolbox, TaskStatus taskStatus) throws Exception + { + super.cleanUp(toolbox, taskStatus); + runComplete.countDown(); + } + + public SegmentIdWithShardSpec allocateOrGetSegmentForTimestamp(String timestamp) + { + final DateTime time = DateTime.parse(timestamp); + for (SegmentIdWithShardSpec pendingSegment : pendingSegments) { + if (pendingSegment.getInterval().contains(time)) { + return pendingSegment; + } + } + return allocateNewSegmentForDate(time); + } + + public SegmentIdWithShardSpec allocateNewSegmentForTimestamp(String timestamp) + { + return allocateNewSegmentForDate(DateTime.parse(timestamp)); + } + + private SegmentIdWithShardSpec allocateNewSegmentForDate(DateTime time) + { + try { + SegmentAllocateAction allocateAction = new SegmentAllocateAction( + getDataSource(), + time, + Granularities.NONE, + segmentGranularity, + getId() + "_" + sequenceId.getAndIncrement(), + null, + false, + NumberedPartialShardSpec.instance(), + LockGranularity.TIME_CHUNK, + TaskLockType.valueOf(lockType) + ); + final SegmentIdWithShardSpec id = toolbox.getTaskActionClient().submit(allocateAction); + pendingSegments.add(id); + return id; + } + catch (Exception e) { + return null; + } + + } + + private Set allocatePendingSegments(TaskToolbox toolbox) throws IOException + { + final Set pendingSegments = new HashSet<>(); + + int sequenceId = 0; + for (int i = 0; i < numPartitions; i++) { + DateTime timestamp = interval.getStart(); + while (true) { + SegmentAllocateAction allocateAction = new SegmentAllocateAction( + getDataSource(), + timestamp, + Granularities.NONE, + new DurationGranularity(interval.getEndMillis() - timestamp.getMillis(), null), + getId() + "_" + sequenceId++, + null, + false, + NumberedPartialShardSpec.instance(), + LockGranularity.TIME_CHUNK, + TaskLockType.valueOf(lockType) + ); + final SegmentIdWithShardSpec id = toolbox.getTaskActionClient().submit(allocateAction); + pendingSegments.add( + id + ); + timestamp = id.getInterval().getEnd(); + if (timestamp.equals(interval.getEnd())) { + break; + } + } + + } + return pendingSegments; + } + + private Set convertPendingSegments(Set pendingSegments) + { + final Set segments = new HashSet<>(); + for (SegmentIdWithShardSpec pendingSegment : pendingSegments) { + final SegmentId id = pendingSegment.asSegmentId(); + segments.add( + new DataSegment( + id, + ImmutableMap.of(id.toString(), id.toString()), + ImmutableList.of(), + ImmutableList.of(), + pendingSegment.getShardSpec(), + null, + 0, + 0 + ) + ); + } + return segments; + } + + private boolean publishSegments(TaskToolbox toolbox, Set newSegments) + throws Exception + { + final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish, commitMetadata) -> + toolbox.getTaskActionClient().submit( + SegmentTransactionalAppendAction.appendAction(segmentsToPublish, null, null) + ); + return publisher.publishSegments( + Collections.emptySet(), + Collections.emptySet(), + newSegments, + Function.identity(), + null + ).isSuccess(); + } + + @Override + public int getPriority() + { + return priority; + } + + public void markReady() + { + readyLatch.countDown(); + } + + public void beginPublish() + { + runLatch.countDown(); + } + + public void awaitReadyComplete() throws InterruptedException + { + readyComplete.await(); + } + + public void completeSegmentAllocation() + { + segmentAllocationComplete.countDown(); + } + + public void awaitRunComplete() throws InterruptedException + { + runComplete.await(); + } + +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index bdfb17ac1786..93a8208399aa 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -277,7 +277,7 @@ public TaskStatus runTask(final TaskToolbox toolbox) DiscoveryDruidNode discoveryDruidNode = createDiscoveryDruidNode(toolbox); appenderator = newAppenderator(dataSchema, tuningConfig, metrics, toolbox); - TaskLockType lockType = getContextValue(Tasks.USE_SHARED_LOCK, false) ? TaskLockType.SHARED : TaskLockType.EXCLUSIVE; + TaskLockType lockType = getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE); StreamAppenderatorDriver driver = newDriver(dataSchema, appenderator, toolbox, metrics, lockType); try { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 5e3d7b65fbfb..12fd603ab68a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -50,10 +50,13 @@ import org.apache.druid.indexer.partitions.SecondaryPartitionType; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; +import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentGenerateTask; @@ -911,9 +914,23 @@ private TaskStatus generateAndPublishSegments( } - final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish, commitMetadata) -> - toolbox.getTaskActionClient() - .submit(SegmentTransactionalInsertAction.overwriteAction(segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish)); + final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish, commitMetadata) -> { + TaskLockType lockType = TaskLockType.valueOf(getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name())); + switch (lockType) { + case REPLACE: + return toolbox.getTaskActionClient().submit( + SegmentTransactionalReplaceAction.overwriteAction(segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish) + ); + case APPEND: + return toolbox.getTaskActionClient().submit( + SegmentTransactionalAppendAction.appendAction(segmentsToPublish, null, null) + ); + default: + return toolbox.getTaskActionClient().submit( + SegmentTransactionalInsertAction.overwriteAction(segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish) + ); + } + }; String effectiveId = getContextValue(CompactionTask.CTX_KEY_APPENDERATOR_TRACKING_TASK_ID, null); if (effectiveId == null) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ReplaceTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ReplaceTask.java new file mode 100644 index 000000000000..5bcfb4ee58e6 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ReplaceTask.java @@ -0,0 +1,256 @@ +/* + * 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.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +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.TaskToolbox; +import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; +import org.apache.druid.indexing.common.actions.SurrogateTaskActionClient; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.overlord.Segments; +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.granularity.Granularity; +import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; +import org.apache.druid.server.security.ResourceAction; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; +import org.joda.time.Interval; + +import javax.annotation.Nonnull; +import java.io.IOException; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.function.Function; + +/** + * A test Task which mimics a replacing task by having similar interactions with the overlord. + * + * Begins running by acquiring a REPLACE lock + * + * Task ends after publishing a set of core partitions and + * creating metadata copies for all appended segments published when this lock was held + */ +public class ReplaceTask extends AbstractTask +{ + private final Interval interval; + private final Granularity segmentGranularity; + private final String lockType; + private final int priority; + private final int numCorePartitions; + private final CountDownLatch readyLatch = new CountDownLatch(1); + private final CountDownLatch readyComplete = new CountDownLatch(1); + private final CountDownLatch runLatch = new CountDownLatch(1); + private final CountDownLatch runComplete = new CountDownLatch(1); + private String version; + + @JsonCreator + public ReplaceTask( + @JsonProperty("id") String id, + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval, + @JsonProperty("segmentGranularity") Granularity segmentGranularity, + @JsonProperty("context") Map context + ) + { + super( + id == null ? StringUtils.format("replace_%s_%s", DateTimes.nowUtc(), UUID.randomUUID().toString()) : id, + dataSource == null ? "none" : dataSource, + context, + IngestionMode.REPLACE + ); + this.interval = interval; + this.segmentGranularity = segmentGranularity; + this.lockType = getContextValue(Tasks.TASK_LOCK_TYPE, "EXCLUSIVE"); + this.priority = getContextValue(Tasks.PRIORITY_KEY, 0); + this.numCorePartitions = getContextValue("numCorePartitions", 0); + } + + @Override + public String getType() + { + return "replace"; + } + + @Nonnull + @JsonIgnore + @Override + public Set getInputSourceResources() + { + return ImmutableSet.of(); + } + + @Override + public String setup(TaskToolbox toolbox) throws Exception + { + readyLatch.await(); + while (!isReady(toolbox.getTaskActionClient())) { + Thread.sleep(100); + } + return null; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + return tryTimeChunkLockSingleInterval( + new SurrogateTaskActionClient(getId(), taskActionClient), + interval, + TaskLockType.valueOf(lockType) + ); + } + + private boolean tryTimeChunkLockSingleInterval(TaskActionClient client, Interval interval, TaskLockType lockType) + throws IOException + { + final TaskLock lock = client.submit(new TimeChunkLockTryAcquireAction(lockType, interval)); + if (lock == null) { + return false; + } + if (lock.isRevoked()) { + throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", interval)); + } + version = lock.getVersion(); + return true; + } + + private Set findSegmentsToLock(TaskActionClient taskActionClient) + throws IOException + { + return ImmutableSet.copyOf( + taskActionClient.submit( + new RetrieveUsedSegmentsAction(getDataSource(), null, ImmutableList.of(interval), Segments.ONLY_VISIBLE) + ) + ); + } + + @Override + public void stopGracefully(TaskConfig taskConfig) + { + } + + @Override + public TaskStatus runTask(TaskToolbox toolbox) throws Exception + { + final Set oldSegments = findSegmentsToLock(toolbox.getTaskActionClient()); + + readyComplete.countDown(); + + runLatch.await(); + + final Set newSegments = createSegments(); + if (publishSegments(toolbox, oldSegments, newSegments)) { + return TaskStatus.success(getId()); + } + return TaskStatus.failure(getId(), "Failed to replace segments"); + } + + @Override + public void cleanUp(TaskToolbox toolbox, TaskStatus taskStatus) throws Exception + { + super.cleanUp(toolbox, taskStatus); + runComplete.countDown(); + } + + + private Set createSegments() + { + final Set newSegments = new HashSet<>(); + for (int i = 0; i < numCorePartitions; i++) { + for (Interval subInterval : segmentGranularity.getIterable(interval)) { + final ShardSpec shardSpec = new NumberedShardSpec(i, numCorePartitions); + final SegmentId segmentId = SegmentId.of(getDataSource(), subInterval, version, shardSpec); + newSegments.add( + new DataSegment( + segmentId, + ImmutableMap.of(segmentId.toString(), segmentId.toString()), + ImmutableList.of(), + ImmutableList.of(), + shardSpec, + null, + 0, + 0 + ) + ); + } + } + return newSegments; + } + + private boolean publishSegments(TaskToolbox toolbox, Set oldSegments, Set newSegments) + throws Exception + { + final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish, commitMetadata) -> + toolbox.getTaskActionClient().submit( + SegmentTransactionalReplaceAction.overwriteAction(segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish) + ); + return publisher.publishSegments( + oldSegments, + Collections.emptySet(), + newSegments, + Function.identity(), + null + ).isSuccess(); + } + + @Override + public int getPriority() + { + return priority; + } + + public void awaitRunComplete() throws InterruptedException + { + runComplete.await(); + } + + public void awaitReadyComplete() throws InterruptedException + { + readyComplete.await(); + } + + public void markReady() + { + readyLatch.countDown(); + } + + public void beginPublish() + { + runLatch.countDown(); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index 1cc2d329c88d..ee20bddfe71c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -79,7 +79,9 @@ @Type(name = "index_realtime", value = RealtimeIndexTask.class), @Type(name = "index_realtime_appenderator", value = AppenderatorDriverRealtimeIndexTask.class), @Type(name = "noop", value = NoopTask.class), - @Type(name = "compact", value = CompactionTask.class) + @Type(name = "compact", value = CompactionTask.class), + @Type(name = "replace", value = ReplaceTask.class), + @Type(name = "append", value = AppendTask.class) }) public interface Task { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskLockHelper.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskLockHelper.java index e27b29fa5a60..1584c88f4df1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskLockHelper.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskLockHelper.java @@ -57,7 +57,7 @@ public class TaskLockHelper private final Map overwritingRootGenPartitions = new HashMap<>(); private final Set lockedExistingSegments = new HashSet<>(); private final boolean useSegmentLock; - private final boolean useSharedLock; + private final TaskLockType taskLockType; @Nullable private Granularity knownSegmentGranularity; @@ -91,10 +91,10 @@ public short getMinorVersionForNewSegments() } } - public TaskLockHelper(boolean useSegmentLock, boolean useSharedLock) + public TaskLockHelper(boolean useSegmentLock, TaskLockType taskLockType) { this.useSegmentLock = useSegmentLock; - this.useSharedLock = useSharedLock; + this.taskLockType = taskLockType; } public boolean isUseSegmentLock() @@ -107,14 +107,9 @@ public LockGranularity getLockGranularityToUse() return useSegmentLock ? LockGranularity.SEGMENT : LockGranularity.TIME_CHUNK; } - public boolean isUseSharedLock() - { - return useSharedLock; - } - public TaskLockType getLockTypeToUse() { - return useSharedLock ? TaskLockType.SHARED : TaskLockType.EXCLUSIVE; + return taskLockType == null ? TaskLockType.EXCLUSIVE : taskLockType; } public boolean hasLockedExistingSegments() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java index 90b752697780..f3f391d4f35e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java @@ -51,8 +51,8 @@ public class Tasks public static final String PRIORITY_KEY = "priority"; public static final String LOCK_TIMEOUT_KEY = "taskLockTimeout"; public static final String FORCE_TIME_CHUNK_LOCK_KEY = "forceTimeChunkLock"; - public static final String USE_SHARED_LOCK = "useSharedLock"; public static final String STORE_EMPTY_COLUMNS_KEY = "storeEmptyColumns"; + public static final String TASK_LOCK_TYPE = "taskLockType"; /** * Context flag denoting if maximum possible values should be used to estimate diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 4f7069ad326a..dad5c514fb34 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -42,9 +42,12 @@ 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.TaskLockType; import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; @@ -1167,10 +1170,23 @@ private void publishSegments( } } - final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish, commitMetadata) -> - toolbox.getTaskActionClient().submit( - SegmentTransactionalInsertAction.overwriteAction(segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish) - ); + final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish, commitMetadata) -> { + TaskLockType lockType = TaskLockType.valueOf(getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name())); + switch (lockType) { + case REPLACE: + return toolbox.getTaskActionClient().submit( + SegmentTransactionalReplaceAction.overwriteAction(segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish) + ); + case APPEND: + return toolbox.getTaskActionClient().submit( + SegmentTransactionalAppendAction.appendAction(segmentsToPublish, null, null) + ); + default: + return toolbox.getTaskActionClient().submit( + SegmentTransactionalInsertAction.overwriteAction(segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish) + ); + } + }; final boolean published = newSegments.isEmpty() || publisher.publishSegments(oldSegments, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java index 7f781e466edd..d72bb8d85007 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java @@ -26,9 +26,11 @@ import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.indexing.common.Counters; +import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskCompleteEvent; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.NonnullPair; @@ -283,7 +285,11 @@ public SegmentIdWithShardSpec allocateNewSegment( NonnullPair findIntervalAndVersion(DateTime timestamp) throws IOException { - return AbstractBatchIndexTask.findIntervalAndVersion(getToolbox(), ingestionSchema, timestamp); + TaskLockType taskLockType = null; + if (getContext().containsKey(Tasks.TASK_LOCK_TYPE)) { + taskLockType = TaskLockType.valueOf(getContext().get(Tasks.TASK_LOCK_TYPE).toString()); + } + return AbstractBatchIndexTask.findIntervalAndVersion(getToolbox(), ingestionSchema, timestamp, taskLockType); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java index 2171d6b38680..9d1ecb52b903 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java @@ -41,7 +41,7 @@ public class LockRequestForNewSegment implements LockRequest private final int priority; private final String sequenceName; @Nullable - private final String previsousSegmentId; + private final String previousSegmentId; private final boolean skipSegmentLineageCheck; private String version; @@ -55,7 +55,7 @@ public LockRequestForNewSegment( PartialShardSpec partialShardSpec, int priority, String sequenceName, - @Nullable String previsousSegmentId, + @Nullable String previousSegmentId, boolean skipSegmentLineageCheck ) { @@ -67,7 +67,7 @@ public LockRequestForNewSegment( this.partialShardSpec = partialShardSpec; this.priority = priority; this.sequenceName = sequenceName; - this.previsousSegmentId = previsousSegmentId; + this.previousSegmentId = previousSegmentId; this.skipSegmentLineageCheck = skipSegmentLineageCheck; } @@ -79,7 +79,7 @@ public LockRequestForNewSegment( Interval interval, PartialShardSpec partialShardSpec, String sequenceName, - @Nullable String previsousSegmentId, + @Nullable String previousSegmentId, boolean skipSegmentLineageCheck ) { @@ -92,7 +92,7 @@ public LockRequestForNewSegment( partialShardSpec, task.getPriority(), sequenceName, - previsousSegmentId, + previousSegmentId, skipSegmentLineageCheck ); } @@ -168,9 +168,9 @@ public String getSequenceName() } @Nullable - public String getPrevisousSegmentId() + public String getPreviousSegmentId() { - return previsousSegmentId; + return previousSegmentId; } public boolean isSkipSegmentLineageCheck() @@ -190,7 +190,7 @@ public String toString() ", partialShardSpec=" + partialShardSpec + ", priority=" + priority + ", sequenceName='" + sequenceName + '\'' + - ", previsousSegmentId='" + previsousSegmentId + '\'' + + ", previsousSegmentId='" + previousSegmentId + '\'' + ", skipSegmentLineageCheck=" + skipSegmentLineageCheck + '}'; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index f84f5f3075f9..cc01d152de4d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ComparisonChain; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.inject.Inject; @@ -749,7 +750,7 @@ private SegmentIdWithShardSpec allocateSegmentId(LockRequestForNewSegment reques return metadataStorageCoordinator.allocatePendingSegment( request.getDataSource(), request.getSequenceName(), - request.getPrevisousSegmentId(), + request.getPreviousSegmentId(), request.getInterval(), request.getPartialShardSpec(), version, @@ -1240,6 +1241,28 @@ Map>>> ge return running; } + public Set getAllReplaceLocksForDatasource(final String datasource) + { + giant.lock(); + try { + final NavigableMap>> dsRunning = running.get(datasource); + if (dsRunning == null) { + return ImmutableSet.of(); + } + return dsRunning.values() + .stream() + .flatMap(map -> map.values().stream()) + .flatMap(Collection::stream) + .map(TaskLockPosse::getTaskLock) + .filter(taskLock -> taskLock.getType().equals(TaskLockType.REPLACE)) + .collect(Collectors.toSet()); + } + finally { + giant.unlock(); + } + } + + /** * Check if the lock for a given request can coexist with a given set of conflicting posses without any revocation. * @param conflictPosses conflict lock posses diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java index 3966df66eb73..3a550b4a0a03 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java @@ -115,6 +115,9 @@ public Interval getInterval() @Override public String getVersion() { + if (lockType.equals(TaskLockType.APPEND) && preferredVersion == null) { + return "1970-01-01T00:00:00.000Z"; + } return preferredVersion == null ? DateTimes.nowUtc().toString() : preferredVersion; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 302099685b4a..79f3a6b607fe 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -106,7 +106,7 @@ public SeekableStreamIndexTask( this.lockGranularityToUse = getContextValue(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK) ? LockGranularity.TIME_CHUNK : LockGranularity.SEGMENT; - this.lockTypeToUse = getContextValue(Tasks.USE_SHARED_LOCK, false) ? TaskLockType.SHARED : TaskLockType.EXCLUSIVE; + this.lockTypeToUse = getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE); } protected static String getFormattedGroupId(String dataSource, String type) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 548d5b6a05b7..8f7af1bb4c4b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -1529,7 +1529,7 @@ public SegmentPublishResult announceHistoricalSegments( DataSourceMetadata endMetadata ) throws IOException { - SegmentPublishResult result = super.announceHistoricalSegments(segments, segmentsToDrop, startMetadata, endMetadata); + SegmentPublishResult result = super.announceHistoricalSegments(segments, segmentsToDrop, startMetadata, endMetadata, null, null, false); Assert.assertFalse( "Segment latch not initialized, did you forget to call expectPublishSegments?", diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java new file mode 100644 index 000000000000..cc04c7dbb376 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java @@ -0,0 +1,1361 @@ +/* + * 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.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexing.common.MultipleFileTaskReportFileWriter; +import org.apache.druid.indexing.common.TaskStorageDirTracker; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.TaskToolboxFactory; +import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.actions.TaskActionClientFactory; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.config.TaskConfigBuilder; +import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.indexing.overlord.TaskQueue; +import org.apache.druid.indexing.overlord.TaskRunner; +import org.apache.druid.indexing.overlord.ThreadingTaskRunner; +import org.apache.druid.indexing.overlord.config.DefaultTaskConfig; +import org.apache.druid.indexing.overlord.config.TaskLockConfig; +import org.apache.druid.indexing.overlord.config.TaskQueueConfig; +import org.apache.druid.indexing.worker.config.WorkerConfig; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.apache.druid.tasklogs.NoopTaskLogs; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collection; +import java.util.TreeSet; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class ConcurrentReplaceAndAppendTest extends IngestionTestBase +{ + private TaskQueue taskQueue; + private TaskActionClient taskActionClient; + + private static final WorkerConfig WORKER_CONFIG = new WorkerConfig().setCapacity(10); + + @Before + public void setup() throws Exception + { + final TaskConfig taskConfig = new TaskConfigBuilder().build(); + final TaskActionClientFactory taskActionClientFactory = createActionClientFactory(); + taskActionClient = taskActionClientFactory.create(NoopTask.create()); + final TaskToolboxFactory toolboxFactory = new TestTaskToolboxFactory(taskConfig, taskActionClientFactory); + final TaskRunner taskRunner = new ThreadingTaskRunner( + toolboxFactory, + taskConfig, + WORKER_CONFIG, + new NoopTaskLogs(), + getObjectMapper(), + new TestAppenderatorsManager(), + new MultipleFileTaskReportFileWriter(), + new DruidNode("middleManager", "host", false, 8091, null, true, false), + TaskStorageDirTracker.fromConfigs(WORKER_CONFIG, taskConfig) + ); + taskQueue = new TaskQueue( + new TaskLockConfig(), + new TaskQueueConfig(null, new Period(0L), null, null), + new DefaultTaskConfig(), + getTaskStorage(), + taskRunner, + taskActionClientFactory, + getLockbox(), + new NoopServiceEmitter() + ); + taskQueue.start(); + } + + @Test + public void test() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023/2024"), + Granularities.YEAR, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023/2024"), + Granularities.YEAR, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + ReplaceTask replaceTask1 = new ReplaceTask( + "replace1", + "DS", + Intervals.of("2023/2024"), + Granularities.YEAR, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + ReplaceTask replaceTask2 = new ReplaceTask( + "replace2", + "DS", + Intervals.of("2023/2024"), + Granularities.YEAR, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + // Create a set of initial segments + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(1, 1); + verifyTaskSuccess(replaceTask0); + + // Append task begins and allocates pending segments + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-01"); + appendTask0.completeSegmentAllocation(); + + // New replace task starts and ends before the appending task finishes + taskQueue.add(replaceTask1); + replaceTask1.markReady(); + replaceTask1.beginPublish(); + replaceTask1.awaitRunComplete(); + verifySegmentCount(2, 1); + verifyTaskSuccess(replaceTask1); + + taskQueue.add(replaceTask2); + replaceTask2.markReady(); + replaceTask2.awaitReadyComplete(); + + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(4, 2); + verifyTaskSuccess(appendTask0); + + replaceTask2.beginPublish(); + replaceTask2.awaitRunComplete(); + verifySegmentCount(6, 2); + verifyTaskSuccess(replaceTask2); + } + + @Test + public void testRRAA_dailyReplaceDailyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(31, 31); + verifyTaskSuccess(replaceTask0); + + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(62, 62); + verifyTaskSuccess(appendTask0); + } + + @Test + public void testRAAR_dailyReplaceDailyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.awaitReadyComplete(); + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(31, 31); + verifyTaskSuccess(appendTask0); + + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(93, 62); + verifyTaskSuccess(replaceTask0); + } + + @Test + public void testRARA_dailyReplaceDailyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.awaitReadyComplete(); + + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(31, 31); + verifyTaskSuccess(replaceTask0); + + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(93, 62); + verifyTaskSuccess(appendTask0); + } + + @Test + public void testARRA_dailyReplaceDailyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(31, 31); + verifyTaskSuccess(replaceTask0); + + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(93, 62); + verifyTaskSuccess(appendTask0); + } + + @Test + public void testARAR_dailyReplaceDailyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.awaitReadyComplete(); + + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(31, 31); + verifyTaskSuccess(appendTask0); + + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(93, 62); + verifyTaskSuccess(replaceTask0); + } + + @Test + public void testAARR_dailyReplaceDailyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(31, 31); + verifyTaskSuccess(appendTask0); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(62, 31); + verifyTaskSuccess(replaceTask0); + } + + + + @Test + public void testRRAA_monthlyReplaceDailyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.MONTH, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(1, 1); + verifyTaskSuccess(replaceTask0); + + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(2, 2); + verifyTaskSuccess(appendTask0); + } + + @Test + public void testRAAR_monthlyReplaceDailyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.MONTH, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.awaitReadyComplete(); + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(31, 31); + verifyTaskSuccess(appendTask0); + + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(63, 32); + verifyTaskSuccess(replaceTask0); + } + + @Test + public void testRARA_monthlyReplaceDailyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.MONTH, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.awaitReadyComplete(); + + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(1, 1); + verifyTaskSuccess(replaceTask0); + + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(63, 32); + verifyTaskSuccess(appendTask0); + } + + @Test + public void testARRA_monthlyReplaceDailyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.MONTH, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(1, 1); + verifyTaskSuccess(replaceTask0); + + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(63, 32); + verifyTaskSuccess(appendTask0); + } + + @Test + public void testARAR_monthlyReplaceDailyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.MONTH, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.awaitReadyComplete(); + + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(31, 31); + verifyTaskSuccess(appendTask0); + + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(63, 32); + verifyTaskSuccess(replaceTask0); + } + + @Test + public void testAARR_monthlyReplaceDailyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.MONTH, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(31, 31); + verifyTaskSuccess(appendTask0); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(32, 1); + verifyTaskSuccess(replaceTask0); + } + + + + @Test + public void testRRAA_dailyReplaceMonthlyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.MONTH, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(31, 31); + verifyTaskSuccess(replaceTask0); + + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(62, 62); + verifyTaskSuccess(appendTask0); + } + + @Test + public void testRAAR_dailyReplaceMonthlyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.MONTH, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.awaitReadyComplete(); + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(1, 1); + verifyTaskSuccess(appendTask0); + + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(1, 1); + verifyTaskFailure(replaceTask0); + } + + @Test + public void testRARA_dailyReplaceMonthlyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.MONTH, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.awaitReadyComplete(); + + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(31, 31); + verifyTaskSuccess(replaceTask0); + + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(31, 31); + verifyTaskFailure(appendTask0); + } + + @Test + public void testARRA_dailyReplaceMonthlyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.MONTH, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(31, 31); + verifyTaskSuccess(replaceTask0); + + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(31, 31); + verifyTaskFailure(appendTask0); + } + + @Test + public void testARAR_dailyReplaceMonthlyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.MONTH, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.awaitReadyComplete(); + + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(1, 1); + verifyTaskSuccess(appendTask0); + + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(1, 1); + verifyTaskFailure(replaceTask0); + } + + @Test + public void testAARR_dailyReplaceMonthlyAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.MONTH, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + for (int i = 1; i <= 9; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); + } + for (int i = 10; i <= 31; i++) { + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); + } + appendTask0.completeSegmentAllocation(); + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(1, 1); + verifyTaskSuccess(appendTask0); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(32, 31); + verifyTaskSuccess(replaceTask0); + } + + @Test + public void testMultipleAppend() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023/2024"), + Granularities.YEAR, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023/2024"), + Granularities.YEAR, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + AppendTask appendTask1 = new AppendTask( + "append1", + "DS", + Intervals.of("2023/2024"), + Granularities.YEAR, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-01"); + appendTask0.completeSegmentAllocation(); + + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.awaitReadyComplete(); + + taskQueue.add(appendTask1); + appendTask1.markReady(); + appendTask1.awaitReadyComplete(); + appendTask1.allocateOrGetSegmentForTimestamp("2023-01-01"); + appendTask1.completeSegmentAllocation(); + appendTask1.beginPublish(); + appendTask1.awaitRunComplete(); + verifySegmentCount(1, 1); + verifyTaskSuccess(appendTask1); + + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(3, 2); + verifyTaskSuccess(replaceTask0); + + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(5, 3); + verifyTaskSuccess(appendTask0); + } + + @Test + public void testMultipleGranularities() throws Exception + { + ReplaceTask replaceTask0 = new ReplaceTask( + "replace0", + "DS", + Intervals.of("2023/2024"), + Granularities.YEAR, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "REPLACE", + "numCorePartitions", 1 + ) + ); + + AppendTask appendTask0 = new AppendTask( + "append0", + "DS", + Intervals.of("2023-01-01/2023-02-01"), + Granularities.DAY, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + AppendTask appendTask1 = new AppendTask( + "append1", + "DS", + Intervals.of("2023-07-01/2024-01-01"), + Granularities.QUARTER, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + AppendTask appendTask2 = new AppendTask( + "append2", + "DS", + Intervals.of("2023-12-01/2024-01-01"), + Granularities.MONTH, + ImmutableMap.of( + Tasks.TASK_LOCK_TYPE, "APPEND", + "numPartitions", 1 + ) + ); + + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-01"); + appendTask0.completeSegmentAllocation(); + + + taskQueue.add(appendTask1); + appendTask1.markReady(); + appendTask1.awaitReadyComplete(); + appendTask1.allocateOrGetSegmentForTimestamp("2023-07-01"); + appendTask1.allocateOrGetSegmentForTimestamp("2023-08-01"); + appendTask1.allocateOrGetSegmentForTimestamp("2023-09-01"); + appendTask1.allocateOrGetSegmentForTimestamp("2023-10-01"); + appendTask1.allocateOrGetSegmentForTimestamp("2023-11-01"); + appendTask1.allocateOrGetSegmentForTimestamp("2023-12-01"); + appendTask1.completeSegmentAllocation(); + appendTask1.beginPublish(); + appendTask1.awaitRunComplete(); + verifySegmentCount(2, 2); + verifyTaskSuccess(appendTask1); + + taskQueue.add(appendTask2); + appendTask2.markReady(); + appendTask2.awaitReadyComplete(); + appendTask2.allocateOrGetSegmentForTimestamp("2023-12-01"); + appendTask2.completeSegmentAllocation(); + + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.awaitReadyComplete(); + + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(3, 3); + verifyTaskSuccess(appendTask0); + + replaceTask0.beginPublish(); + replaceTask0.awaitRunComplete(); + verifySegmentCount(5, 2); + verifyTaskSuccess(replaceTask0); + + appendTask2.beginPublish(); + appendTask2.awaitRunComplete(); + verifySegmentCount(7, 3); + verifyTaskSuccess(appendTask2); + } + + private void verifyTaskSuccess(Task task) + { + try { + while (!getTaskStorage().getStatus(task.getId()).get().isComplete()) { + Thread.sleep(100); + } + } + catch (InterruptedException e) { + // do nothing + } + Assert.assertTrue(getTaskStorage().getStatus(task.getId()).get().isSuccess()); + } + + private void verifyTaskFailure(Task task) + { + try { + while (!getTaskStorage().getStatus(task.getId()).get().isComplete()) { + Thread.sleep(100); + } + } + catch (InterruptedException e) { + // do nothing + } + Assert.assertTrue(getTaskStorage().getStatus(task.getId()).get().isFailure()); + } + + private void verifySegmentCount(int expectedTotal, int expectedVisible) throws Exception + { + Collection allUsed = taskActionClient.submit( + new RetrieveUsedSegmentsAction("DS", null, ImmutableList.of(Intervals.ETERNITY), Segments.INCLUDING_OVERSHADOWED) + ); + System.out.println("All used segments: " + allUsed.size()); + System.out.println(new TreeSet<>(allUsed.stream().map(s -> s.getId().toString()).collect(Collectors.toSet()))); + Collection visibleUsed = taskActionClient.submit( + new RetrieveUsedSegmentsAction("DS", null, ImmutableList.of(Intervals.ETERNITY), Segments.ONLY_VISIBLE) + ); + Assert.assertEquals(expectedTotal, allUsed.size()); + System.out.println("All visible segments: " + visibleUsed.size()); + System.out.println(new TreeSet<>(visibleUsed.stream().map(s -> s.getId().toString()).collect(Collectors.toSet()))); + Assert.assertEquals(expectedVisible, visibleUsed.size()); + } + + private class TestTaskToolboxFactory extends TaskToolboxFactory + { + private final TaskConfig taskConfig; + + public TestTaskToolboxFactory(TaskConfig taskConfig, TaskActionClientFactory taskActionClientFactory) + { + super( + taskConfig, + null, + taskActionClientFactory, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + new IndexIO(getObjectMapper(), () -> 0), + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + this.taskConfig = taskConfig; + } + + @Override + public TaskToolbox build(Task task) + { + return build(taskConfig, task); + } + + @Override + public TaskToolbox build(Function decoratorFn, Task task) + { + return build(decoratorFn.apply(taskConfig), task); + } + + + @Override + public TaskToolbox build(TaskConfig config, Task task) + { + try { + return createTaskToolbox(config, task); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 645558c441b1..474ca004e805 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -107,19 +107,20 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private final TestUtils testUtils = new TestUtils(); + protected final TestUtils testUtils = new TestUtils(); private final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); private SegmentCacheManagerFactory segmentCacheManagerFactory; private TaskStorage taskStorage; private IndexerSQLMetadataStorageCoordinator storageCoordinator; private SegmentsMetadataManager segmentsMetadataManager; private TaskLockbox lockbox; + private File baseDir; @Before public void setUpIngestionTestBase() throws IOException { EmittingLogger.registerEmitter(new NoopServiceEmitter()); - temporaryFolder.create(); + baseDir = temporaryFolder.newFolder("baseWorkDir"); final SQLMetadataConnector connector = derbyConnectorRule.getConnector(); connector.createTaskTables(); @@ -225,6 +226,30 @@ public TaskActionToolbox createTaskActionToolbox() ); } + public TaskToolbox createTaskToolbox(TaskConfig config, Task task) throws IOException + { + return new TaskToolbox.Builder() + .config(config) + .taskExecutorNode(new DruidNode("druid/middlemanager", "localhost", false, 8091, null, true, false)) + .taskActionClient(createActionClient(task)) + .segmentPusher(new LocalDataSegmentPusher(new LocalDataSegmentPusherConfig())) + .dataSegmentKiller(new NoopDataSegmentKiller()) + .joinableFactory(NoopJoinableFactory.INSTANCE) + .jsonMapper(objectMapper) + .taskWorkDir(baseDir) + .indexIO(getIndexIO()) + .indexMergerV9(testUtils.getIndexMergerV9Factory() + .create(task.getContextValue(Tasks.STORE_EMPTY_COLUMNS_KEY, true))) + .taskReportFileWriter(new NoopTestTaskReportFileWriter()) + .authorizerMapper(AuthTestUtils.TEST_AUTHORIZER_MAPPER) + .chatHandlerProvider(new NoopChatHandlerProvider()) + .rowIngestionMetersFactory(testUtils.getRowIngestionMetersFactory()) + .appenderatorsManager(new TestAppenderatorsManager()) + .taskLogPusher(null) + .attemptId("1") + .build(); + } + public IndexIO getIndexIO() { return testUtils.getTestIndexIO(); @@ -383,7 +408,7 @@ public ListenableFuture run(Task task) .dataSegmentKiller(new NoopDataSegmentKiller()) .joinableFactory(NoopJoinableFactory.INSTANCE) .jsonMapper(objectMapper) - .taskWorkDir(temporaryFolder.newFolder()) + .taskWorkDir(baseDir) .indexIO(getIndexIO()) .indexMergerV9(testUtils.getIndexMergerV9Factory() .create(task.getContextValue(Tasks.STORE_EMPTY_COLUMNS_KEY, true))) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index d22769c1f791..c5d9482a3d33 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -29,6 +29,7 @@ import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.Tasks; @@ -620,9 +621,9 @@ public void testMultipleAppends() final ParallelIndexSupervisorTask task = newTask(interval, Granularities.DAY, true, true); final ParallelIndexSupervisorTask task2 = newTask(interval, Granularities.DAY, true, true); task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); - task.addToContext(Tasks.USE_SHARED_LOCK, true); + task.addToContext(Tasks.TASK_LOCK_TYPE, TaskLockType.SHARED); task2.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); - task2.addToContext(Tasks.USE_SHARED_LOCK, true); + task2.addToContext(Tasks.TASK_LOCK_TYPE, TaskLockType.SHARED); getIndexingServiceClient().runTask(task.getId(), task); getIndexingServiceClient().runTask(task2.getId(), task2); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index d64bd1d22263..c3038aaedbe8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -28,6 +28,7 @@ import org.apache.druid.indexing.overlord.SegmentCreateRequest; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.indexing.overlord.TaskLockInfo; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Pair; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; @@ -143,13 +144,27 @@ public SegmentPublishResult announceHistoricalSegments( Set segments, Set segmentsToDrop, DataSourceMetadata oldCommitMetadata, - DataSourceMetadata newCommitMetadata + DataSourceMetadata newCommitMetadata, + @Nullable Map segmentLockMap, + @Nullable Set taskLockInfos, + boolean append ) { // Don't actually compare metadata, just do it! return SegmentPublishResult.ok(announceHistoricalSegments(segments)); } + @Override + public SegmentPublishResult announceHistoricalSegments( + Set segments, + Set segmentsToDrop, + @Nullable DataSourceMetadata startMetadata, + @Nullable DataSourceMetadata endMetadata + ) + { + return announceHistoricalSegments(segments, segmentsToDrop, startMetadata, endMetadata, null, null, false); + } + @Override public SegmentPublishResult commitMetadataOnly( String dataSource, diff --git a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java index 45fb6639082c..932115e879e3 100644 --- a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java @@ -77,6 +77,8 @@ default void exportTable( void createSegmentTable(); + void createSegmentVersionTable(); + void createRulesTable(); void createConfigTable(); diff --git a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java index 267f4fe6b69c..bc3203b93c4e 100644 --- a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java +++ b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java @@ -34,7 +34,7 @@ public class MetadataStorageTablesConfig public static MetadataStorageTablesConfig fromBase(String base) { - return new MetadataStorageTablesConfig(base, null, null, null, null, null, null, null, null, null, null); + return new MetadataStorageTablesConfig(base, null, null, null, null, null, null, null, null, null, null, null); } public static final String TASK_ENTRY_TYPE = "task"; @@ -57,6 +57,9 @@ public static MetadataStorageTablesConfig fromBase(String base) @JsonProperty("segments") private final String segmentsTable; + @JsonProperty("segmentVersions") + private final String segmentVersionsTable; + @JsonProperty("rules") private final String rulesTable; @@ -90,13 +93,15 @@ public MetadataStorageTablesConfig( @JsonProperty("taskLog") String taskLogTable, @JsonProperty("taskLock") String taskLockTable, @JsonProperty("audit") String auditTable, - @JsonProperty("supervisors") String supervisorTable + @JsonProperty("supervisors") String supervisorTable, + @JsonProperty("segmentVersions") String segmentVersionsTable ) { this.base = (base == null) ? DEFAULT_BASE : base; this.dataSourceTable = makeTableName(dataSourceTable, "dataSource"); this.pendingSegmentsTable = makeTableName(pendingSegmentsTable, "pendingSegments"); this.segmentsTable = makeTableName(segmentsTable, "segments"); + this.segmentVersionsTable = makeTableName(segmentVersionsTable, "segmentVersions"); this.rulesTable = makeTableName(rulesTable, "rules"); this.configTable = makeTableName(configTable, "config"); @@ -142,6 +147,11 @@ public String getSegmentsTable() return segmentsTable; } + public String getSegmentVersionsTable() + { + return segmentVersionsTable; + } + public String getRulesTable() { return rulesTable; diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index b3c70f0cdbe9..f176d1abca1f 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -251,6 +251,10 @@ SegmentIdWithShardSpec allocatePendingSegment( * @param endMetadata dataSource metadata post-insert will have this endMetadata merged in with * {@link DataSourceMetadata#plus(DataSourceMetadata)}. If null, this insert will not * involve a metadata transaction + * @param segmentLockMap map of segments appended with an append lock to the task lock of the exclusive lock + * held for the datasource + interval during commit + * @param taskLockInfos Set of task lock infos for which new segments with given replace the old ones + * for the datasource + interval during commit * * @return segment publish result indicating transaction success or failure, and set of segments actually published. * This method must only return a failure code if it is sure that the transaction did not happen. If it is not sure, @@ -259,6 +263,16 @@ SegmentIdWithShardSpec allocatePendingSegment( * @throws IllegalArgumentException if startMetadata and endMetadata are not either both null or both non-null * @throws RuntimeException if the state of metadata storage after this call is unknown */ + SegmentPublishResult announceHistoricalSegments( + Set segments, + Set segmentsToDrop, + @Nullable DataSourceMetadata startMetadata, + @Nullable DataSourceMetadata endMetadata, + @Nullable Map segmentLockMap, + @Nullable Set taskLockInfos, + boolean append + ) throws IOException; + SegmentPublishResult announceHistoricalSegments( Set segments, Set segmentsToDrop, diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/TaskLockInfo.java b/server/src/main/java/org/apache/druid/indexing/overlord/TaskLockInfo.java new file mode 100644 index 000000000000..a6ea0fad0066 --- /dev/null +++ b/server/src/main/java/org/apache/druid/indexing/overlord/TaskLockInfo.java @@ -0,0 +1,66 @@ +/* + * 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.overlord; + +import org.joda.time.Interval; + +import java.util.Objects; + +public class TaskLockInfo +{ + private final Interval interval; + private final String version; + + public TaskLockInfo(Interval interval, String version) + { + this.interval = interval; + this.version = version; + } + + public Interval getInterval() + { + return interval; + } + + public String getVersion() + { + return version; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TaskLockInfo that = (TaskLockInfo) o; + return Objects.equals(interval, that.getInterval()) && + Objects.equals(version, that.version); + } + + @Override + public int hashCode() + { + return Objects.hash(interval, version); + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 307bfb0508d9..e5cdbd7c2ad5 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -38,6 +38,7 @@ import org.apache.druid.indexing.overlord.SegmentCreateRequest; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.indexing.overlord.TaskLockInfo; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -55,10 +56,14 @@ import org.apache.druid.timeline.SegmentTimeline; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.partition.NoneShardSpec; +import org.apache.druid.timeline.partition.NumberedPartialShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartialShardSpec; import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.PartitionIds; +import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.chrono.ISOChronology; @@ -123,6 +128,7 @@ public void start() connector.createDataSourceTable(); connector.createPendingSegmentsTable(); connector.createSegmentTable(); + connector.createSegmentVersionTable(); } @Override @@ -234,14 +240,14 @@ private Set getPendingSegmentsForIntervalWithHandle( final ResultIterator dbSegments = handle.createQuery( - StringUtils.format( - // This query might fail if the year has a different number of digits - // See https://github.com/apache/druid/pull/11582 for a similar issue - // Using long for these timestamps instead of varchar would give correct time comparisons - "SELECT payload FROM %1$s WHERE dataSource = :dataSource AND start < :end and %2$send%2$s > :start", - dbTables.getPendingSegmentsTable(), connector.getQuoteString() - ) - ) + StringUtils.format( + // This query might fail if the year has a different number of digits + // See https://github.com/apache/druid/pull/11582 for a similar issue + // Using long for these timestamps instead of varchar would give correct time comparisons + "SELECT payload FROM %1$s WHERE dataSource = :dataSource AND start < :end and %2$send%2$s > :start", + dbTables.getPendingSegmentsTable(), connector.getQuoteString() + ) + ) .bind("dataSource", dataSource) .bind("start", interval.getStart().toString()) .bind("end", interval.getEnd().toString()) @@ -293,7 +299,7 @@ private Collection retrieveAllUsedSegmentsForIntervalsWithHandle( @Override public Set announceHistoricalSegments(final Set segments) throws IOException { - final SegmentPublishResult result = announceHistoricalSegments(segments, null, null, null); + final SegmentPublishResult result = announceHistoricalSegments(segments, null, null, null, null, null, false); // Metadata transaction cannot fail because we are not trying to do one. if (!result.isSuccess()) { @@ -308,7 +314,10 @@ public SegmentPublishResult announceHistoricalSegments( final Set segments, final Set segmentsToDrop, @Nullable final DataSourceMetadata startMetadata, - @Nullable final DataSourceMetadata endMetadata + @Nullable DataSourceMetadata endMetadata, + @Nullable Map segmentLockMap, + @Nullable Set taskLockInfos, + boolean append ) throws IOException { if (segments.isEmpty()) { @@ -327,15 +336,57 @@ public SegmentPublishResult announceHistoricalSegments( } // Find which segments are used (i.e. not overshadowed). - final Set usedSegments = new HashSet<>(); - List> segmentHolders = - SegmentTimeline.forSegments(segments).lookupWithIncompletePartitions(Intervals.ETERNITY); - for (TimelineObjectHolder holder : segmentHolders) { - for (PartitionChunk chunk : holder.getObject()) { - usedSegments.add(chunk.getObject()); + Set usedSegments = new HashSet<>(); + Set newSegments = new HashSet<>(segments); + if (!append) { + List> segmentHolders = + SegmentTimeline.forSegments(segments).lookupWithIncompletePartitions(Intervals.ETERNITY); + for (TimelineObjectHolder holder : segmentHolders) { + for (PartitionChunk chunk : holder.getObject()) { + usedSegments.add(chunk.getObject()); + } } + } else { + final Map> segmentToNewMetadataMap = connector.retryTransaction( + new TransactionCallback>>() + { + @Override + public Map> inTransaction( + final Handle handle, + final TransactionStatus transactionStatus + ) throws Exception + { + return allocateNewSegmentIds( + handle, + dataSource, + segments + ); + } + }, + 0, + SQLMetadataConnector.DEFAULT_MAX_TRIES + ); + for (DataSegment segment : segmentToNewMetadataMap.keySet()) { + for (SegmentIdWithShardSpec newId : segmentToNewMetadataMap.get(segment)) { + DataSegment newSegment = new DataSegment( + newId.getDataSource(), + newId.getInterval(), + newId.getVersion(), + segment.getLoadSpec(), + segment.getDimensions(), + segment.getMetrics(), + newId.getShardSpec(), + segment.getBinaryVersion(), + segment.getSize() + ); + newSegments.add(newSegment); + } + } + usedSegments.addAll(newSegments); } + + final AtomicBoolean definitelyNotUpdated = new AtomicBoolean(false); try { @@ -391,7 +442,7 @@ public SegmentPublishResult inTransaction( } } - final Set inserted = announceHistoricalSegmentBatch(handle, segments, usedSegments); + final Set inserted = announceHistoricalSegmentBatch(handle, newSegments, usedSegments, segmentLockMap, taskLockInfos, append); return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); } @@ -410,6 +461,17 @@ public SegmentPublishResult inTransaction( } } + @Override + public SegmentPublishResult announceHistoricalSegments( + Set segments, + Set segmentsToDrop, + @Nullable DataSourceMetadata startMetadata, + @Nullable DataSourceMetadata endMetadata + ) throws IOException + { + return announceHistoricalSegments(segments, segmentsToDrop, startMetadata, endMetadata, null, null, false); + } + @Override public SegmentPublishResult commitMetadataOnly( String dataSource, @@ -988,6 +1050,118 @@ private void insertPendingSegmentIntoMetastore( .execute(); } + private Map> allocateNewSegmentIds( + Handle handle, + String dataSource, + Set segments + ) throws IOException + { + if (segments.isEmpty()) { + return Collections.emptyMap(); + } + + // Map from version to used committed segments + Map> versionToSegments = new HashMap<>(); + Collection segmentIntervals = segments.stream() + .map(DataSegment::getInterval) + .collect(Collectors.toSet()); + try (final CloseableIterator iterator = + SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper) + .retrieveUsedSegments( + dataSource, + segmentIntervals + ) + ) { + while (iterator.hasNext()) { + final DataSegment segment = iterator.next(); + versionToSegments.computeIfAbsent(segment.getVersion(), v -> new HashSet<>()) + .add(segment); + } + } + + // Maps segment to its new metadata with higher versions + Map> retVal = new HashMap<>(); + for (DataSegment segment : segments) { + retVal.put(segment, new HashSet<>()); + } + + for (String version : versionToSegments.keySet()) { + Set lowerVersionSegments = new HashSet<>(); + for (DataSegment segment : segments) { + if (segment.getVersion().compareTo(version) < 0) { + lowerVersionSegments.add(segment); + } + } + + Map> intervalToSegments = new HashMap<>(); + for (DataSegment segment : lowerVersionSegments) { + for (Interval interval : intervalToSegments.keySet()) { + if (interval.contains(segment.getInterval())) { + intervalToSegments.get(interval).add(segment); + break; + } + } + Collection overlappingSegments = retrieveUsedSegmentsForIntervals( + segment.getDataSource(), + ImmutableList.of(segment.getInterval()), + Segments.ONLY_VISIBLE + ); + for (DataSegment overlappingSegment : overlappingSegments) { + if (overlappingSegment.getInterval().contains(segment.getInterval())) { + intervalToSegments.computeIfAbsent(overlappingSegment.getInterval(), itvl -> new HashSet<>()) + .add(segment); + } else { + throw new ISE( + "Existing segment interval[%s] conflicts with that of the new segment[%s]", + overlappingSegment.getInterval(), + segment.getInterval() + ); + } + } + } + + for (Interval interval : intervalToSegments.keySet()) { + Set pendingSegments = new HashSet<>( + getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval) + ); + Collection committedSegments = retrieveAllUsedSegmentsForIntervalsWithHandle( + handle, + dataSource, + ImmutableList.of(interval) + ) + .stream() + .filter(s -> s.getVersion().equals(version) && s.getInterval().equals(interval)) + .collect(Collectors.toSet()); + SegmentIdWithShardSpec committedMaxId = null; + for (DataSegment committedSegment : committedSegments) { + if (committedMaxId == null + || committedMaxId.getShardSpec().getPartitionNum() < committedSegment.getShardSpec().getPartitionNum()) { + committedMaxId = SegmentIdWithShardSpec.fromDataSegment(committedSegment); + } + } + for (DataSegment segment : intervalToSegments.get(interval)) { + SegmentCreateRequest request = new SegmentCreateRequest( + segment.getId().toString() + version, + null, + version, + NumberedPartialShardSpec.instance() + ); + SegmentIdWithShardSpec newId = createNewSegment( + request, + dataSource, + interval, + version, + committedMaxId, + pendingSegments + ); + pendingSegments.add(newId); + retVal.get(segment).add(newId); + } + } + } + return retVal; + } + private Map createNewSegments( Handle handle, String dataSource, @@ -1149,8 +1323,8 @@ private SegmentIdWithShardSpec createNewSegment( version, partialShardSpec.complete(jsonMapper, newPartitionId, 0) ); - } else if (!overallMaxId.getInterval().equals(interval) - || overallMaxId.getVersion().compareTo(existingVersion) > 0) { + + } else if (!overallMaxId.getInterval().equals(interval)) { log.warn( "Cannot allocate new segment for dataSource[%s], interval[%s], existingVersion[%s]: conflicting segment[%s].", dataSource, @@ -1308,8 +1482,7 @@ private SegmentIdWithShardSpec createNewSegment( version, partialShardSpec.complete(jsonMapper, newPartitionId, 0) ); - } else if (!overallMaxId.getInterval().equals(interval) - || overallMaxId.getVersion().compareTo(existingVersion) > 0) { + } else if (!overallMaxId.getInterval().equals(interval)) { log.warn( "Cannot allocate new segment for dataSource[%s], interval[%s], existingVersion[%s]: conflicting segment[%s].", dataSource, @@ -1386,7 +1559,10 @@ public int deletePendingSegments(String dataSource) private Set announceHistoricalSegmentBatch( final Handle handle, final Set segments, - final Set usedSegments + final Set usedSegments, + @Nullable Map appendSegmentLockMap, + @Nullable Set replaceLocks, + boolean append ) throws IOException { final Set toInsertSegments = new HashSet<>(); @@ -1410,49 +1586,254 @@ private Set announceHistoricalSegmentBatch( PreparedBatch preparedBatch = handle.prepareBatch( StringUtils.format( "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", dbTables.getSegmentsTable(), connector.getQuoteString() ) ); - for (List partition : partitionedSegments) { for (DataSegment segment : partition) { preparedBatch.add() - .bind("id", segment.getId().toString()) - .bind("dataSource", segment.getDataSource()) - .bind("created_date", DateTimes.nowUtc().toString()) - .bind("start", segment.getInterval().getStart().toString()) - .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) - .bind("version", segment.getVersion()) - .bind("used", usedSegments.contains(segment)) - .bind("payload", jsonMapper.writeValueAsBytes(segment)); + .bind("id", segment.getId().toString()) + .bind("dataSource", segment.getDataSource()) + .bind("created_date", DateTimes.nowUtc().toString()) + .bind("start", segment.getInterval().getStart().toString()) + .bind("end", segment.getInterval().getEnd().toString()) + .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) + .bind("version", segment.getVersion()) + .bind("used", usedSegments.contains(segment)) + .bind("payload", jsonMapper.writeValueAsBytes(segment)); } - final int[] affectedRows = preparedBatch.execute(); - final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); + final int[] affectedInsertRows = preparedBatch.execute(); + + final boolean succeeded = Arrays.stream(affectedInsertRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); if (succeeded) { log.infoSegments(partition, "Published segments to DB"); } else { final List failedToPublish = IntStream.range(0, partition.size()) - .filter(i -> affectedRows[i] != 1) - .mapToObj(partition::get) - .collect(Collectors.toList()); + .filter(i -> affectedInsertRows[i] != 1) + .mapToObj(partition::get) + .collect(Collectors.toList()); throw new ISE( "Failed to publish segments to DB: %s", SegmentUtils.commaSeparatedIdentifiers(failedToPublish) ); } } + + PreparedBatch appendBatch = handle.prepareBatch( + StringUtils.format( + "INSERT INTO %1$s (id, dataSource, start, %2$send%2$s, segment_id, lock_version) " + + "VALUES (:id, :dataSource, :start, :end, :segment_id, :lock_version)", + dbTables.getSegmentVersionsTable(), + connector.getQuoteString() + ) + ); + if (appendSegmentLockMap == null) { + appendSegmentLockMap = new HashMap<>(); + } + final List>> appendSegmentPartitions = Lists.partition( + new ArrayList<>(appendSegmentLockMap.entrySet()), + MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE + ); + for (List> partition : appendSegmentPartitions) { + for (Map.Entry entry : partition) { + DataSegment segment = entry.getKey(); + TaskLockInfo lock = entry.getValue(); + appendBatch.add() + .bind("id", segment.getId().toString() + ":" + lock.hashCode()) + .bind("dataSource", segment.getDataSource()) + .bind("start", lock.getInterval().getStartMillis()) + .bind("end", lock.getInterval().getEndMillis()) + .bind("segment_id", segment.getId().toString()) + .bind("lock_version", lock.getVersion()); + } + final int[] affectedAppendRows = appendBatch.execute(); + final boolean succeeded = Arrays.stream(affectedAppendRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); + if (!succeeded) { + final List failedToForward = IntStream.range(0, partition.size()) + .filter(i -> affectedAppendRows[i] != 1) + .mapToObj(partition::get) + .map(x -> x.getKey()) + .collect(Collectors.toList()); + throw new ISE( + "Failed to forward appended segments to DB: %s", + SegmentUtils.commaSeparatedIdentifiers(failedToForward) + ); + } + } + + Map segmentsToBeForwarded = new HashMap<>(); + if (!append) { + segmentsToBeForwarded = getAppendedSegmentIds( + handle, + segments.iterator().next().getDataSource(), + replaceLocks + ); + } + final int numCorePartitions = segments.size(); + int partitionNum = segments.size(); + final List>> forwardSegmentsBatch = Lists.partition( + new ArrayList<>(segmentsToBeForwarded.entrySet()), + MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE + ); + for (List> batch : forwardSegmentsBatch) { + Map batchMap = new HashMap<>(); + for (Map.Entry entry : batch) { + batchMap.put(entry.getKey(), entry.getValue()); + } + List oldSegments = retrieveSegments(handle, batchMap.keySet()); + for (DataSegment oldSegment : oldSegments) { + Interval newInterval = oldSegment.getInterval(); + for (DataSegment segment : segments) { + if (segment.getInterval().overlaps(newInterval)) { + if (segment.getInterval().contains(newInterval)) { + newInterval = segment.getInterval(); + } else { + throw new ISE("Incompatible segment intervals for commit: [%s] and [%s].", + newInterval, + segment.getInterval() + ); + } + } + } + TaskLockInfo lock = batchMap.get(oldSegment.getId().toString()); + ShardSpec shardSpec = new NumberedShardSpec(partitionNum++, numCorePartitions); + DataSegment newSegment = new DataSegment( + oldSegment.getDataSource(), + newInterval, + lock.getVersion(), + oldSegment.getLoadSpec(), + oldSegment.getDimensions(), + oldSegment.getMetrics(), + shardSpec, + oldSegment.getBinaryVersion(), + oldSegment.getSize() + ); + preparedBatch.add() + .bind("id", newSegment.getId().toString()) + .bind("dataSource", newSegment.getDataSource()) + .bind("created_date", DateTimes.nowUtc().toString()) + .bind("start", newSegment.getInterval().getStart().toString()) + .bind("end", newSegment.getInterval().getEnd().toString()) + .bind("partitioned", (newSegment.getShardSpec() instanceof NoneShardSpec) ? false : true) + .bind("version", newSegment.getVersion()) + .bind("used", true) + .bind("payload", jsonMapper.writeValueAsBytes(newSegment)); + } + final int[] affectedInsertRows = preparedBatch.execute(); + + final boolean succeeded = Arrays.stream(affectedInsertRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); + if (succeeded) { + log.info("Published segments with updated metadata to DB"); + } else { + throw new ISE("Failed to update segment metadatas in DB"); + } + } } catch (Exception e) { - log.errorSegments(segments, "Exception inserting segments"); + log.errorSegments(segments, "Exception inserting segment metadata"); throw e; } return toInsertSegments; } + private List retrieveSegments(final Handle handle, final Set segmentIds) + { + final StringBuilder sb = new StringBuilder(); + sb.append("SELECT payload FROM %s WHERE id in ("); + + List segmentIdList = new ArrayList<>(segmentIds); + int n = segmentIdList.size(); + for (int i = 0; i < n; i++) { + sb.append("'"); + sb.append(segmentIdList.get(i)); + sb.append("'"); + if (i < n - 1) { + sb.append(", "); + } + } + + sb.append(")"); + + final Query> sql = handle + .createQuery(StringUtils.format(sb.toString(), dbTables.getSegmentsTable())) + .setFetchSize(connector.getStreamingFetchSize()); + + final ResultIterator resultIterator = + sql.map((index, r, ctx) -> JacksonUtils.readValue(jsonMapper, r.getBytes(1), DataSegment.class)) + .iterator(); + + List retVal = new ArrayList<>(); + while (resultIterator.hasNext()) { + retVal.add(resultIterator.next()); + } + return retVal; + } + + private Map getAppendedSegmentIds( + Handle handle, + String datasource, + Set replaceLocks + ) + { + if (CollectionUtils.isNullOrEmpty(replaceLocks)) { + return Collections.emptyMap(); + } + final StringBuilder sb = new StringBuilder(); + sb.append( + StringUtils.format( + "SELECT segment_id, start, %1$send%1$s, lock_version FROM %2$s where dataSource = :dataSource AND (", + connector.getQuoteString(), + dbTables.getSegmentVersionsTable() + ) + ); + + List locks = new ArrayList<>(replaceLocks); + int n = locks.size(); + for (int i = 0; i < n; i++) { + sb.append( + StringUtils.format( + "(start = %2$s AND %1$send%1$s = %3$s AND lock_version = %4$s)", + connector.getQuoteString(), + StringUtils.format(":start%d", i), + StringUtils.format(":end%d", i), + StringUtils.format(":lock_version%d", i) + ) + ); + if (i < n - 1) { + sb.append(" OR "); + } + } + + sb.append(")"); + + Query> query = handle + .createQuery( + sb.toString() + ) + .bind("dataSource", datasource); + for (int i = 0; i < n; i++) { + query.bind(StringUtils.format("start%d", i), locks.get(i).getInterval().getStartMillis()) + .bind(StringUtils.format("end%d", i), locks.get(i).getInterval().getEndMillis()) + .bind(StringUtils.format("lock_version%d", i), locks.get(i).getVersion()); + } + + final ResultIterator> resultIterator = query.map((index, r, ctx) -> { + String segmentId = r.getString("segment_id"); + Interval interval = new Interval(r.getLong("start"), r.getLong("end")); + String version = r.getString("lock_version"); + return Pair.of(segmentId, new TaskLockInfo(interval, version)); + }).iterator(); + Map retVal = new HashMap<>(); + while (resultIterator.hasNext()) { + Pair result = resultIterator.next(); + retVal.put(result.lhs, result.rhs); + } + return retVal; + } + private Set segmentExistsBatch(final Handle handle, final Set segments) { Set existedSegments = new HashSet<>(); @@ -1521,7 +1902,7 @@ private Set segmentExistsBatch(final Handle handle, final Set Date: Wed, 2 Aug 2023 10:00:17 +0530 Subject: [PATCH 02/43] Fix compile errors --- .../apache/druid/metadata/TestMetadataStorageConnector.java | 6 ++++++ .../druid/metadata/TestMetadataStorageTablesConfig.java | 1 + 2 files changed, 7 insertions(+) diff --git a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java index 028a9d5cc08c..6edda9949957 100644 --- a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java +++ b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java @@ -54,6 +54,12 @@ public void createSegmentTable() throw new UnsupportedOperationException(); } + @Override + public void createSegmentVersionTable() + { + throw new UnsupportedOperationException(); + } + @Override public void createRulesTable() { diff --git a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java index 693b36ce24e8..8a864f425548 100644 --- a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java +++ b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java @@ -37,6 +37,7 @@ public TestMetadataStorageTablesConfig() null, null, null, + null, null ); } From 5acf93b5382aa69bb155dd61b8cc5f643cc1b7fe Mon Sep 17 00:00:00 2001 From: Amatya Date: Wed, 2 Aug 2023 10:18:15 +0530 Subject: [PATCH 03/43] Fix compilation in test --- .../indexing/common/task/ConcurrentReplaceAndAppendTest.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java index cc04c7dbb376..f38dab1c2541 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java @@ -41,6 +41,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.server.DruidNode; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.tasklogs.NoopTaskLogs; @@ -83,7 +84,7 @@ public void setup() throws Exception ); taskQueue = new TaskQueue( new TaskLockConfig(), - new TaskQueueConfig(null, new Period(0L), null, null), + new TaskQueueConfig(null, new Period(0L), null, null, null), new DefaultTaskConfig(), getTaskStorage(), taskRunner, @@ -1309,7 +1310,7 @@ public TestTaskToolboxFactory(TaskConfig taskConfig, TaskActionClientFactory tas null, null, null, - new IndexIO(getObjectMapper(), () -> 0), + new IndexIO(getObjectMapper(), ColumnConfig.DEFAULT), null, null, null, From be23936027c8ffd4c03e6847503c4640683c1be2 Mon Sep 17 00:00:00 2001 From: Amatya Date: Thu, 3 Aug 2023 11:56:16 +0530 Subject: [PATCH 04/43] Clean stray comments and move test tasks under test --- .../common/actions/SegmentInsertAction.java | 3 - .../indexing/common/actions/TaskLocks.java | 1 - .../druid/indexing/common/task/Task.java | 2 - .../druid/indexing/overlord/TaskLockbox.java | 18 +- .../task/ConcurrentReplaceAndAppendTest.java | 285 ++++++------------ .../task/batch/parallel}/AppendTask.java | 49 +-- .../task/batch/parallel}/ReplaceTask.java | 32 +- .../TransactionalSegmentPublisher.java | 5 - 8 files changed, 135 insertions(+), 260 deletions(-) rename indexing-service/src/{main/java/org/apache/druid/indexing/common/task => test/java/org/apache/druid/indexing/common/task/batch/parallel}/AppendTask.java (86%) rename indexing-service/src/{main/java/org/apache/druid/indexing/common/task => test/java/org/apache/druid/indexing/common/task/batch/parallel}/ReplaceTask.java (90%) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java index b8f27389aaca..b81d3d377047 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java @@ -30,9 +30,6 @@ import java.util.Set; /** - * Insert segments into metadata storage. The segment versions must all be less than or equal to a lock held by - * your task for the segment intervals. - *

* Word of warning: Very large "segments" sets can cause oversized audit log entries, which is bad because it means * that the task cannot actually complete. Callers should avoid this by avoiding inserting too many segments in the * same action. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java index ab4d0a981a16..400d0f38fdc3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java @@ -123,7 +123,6 @@ public static Set findReplaceLocksForSegments( final Set found = new HashSet<>(); final Set locks = taskLockbox.getAllReplaceLocksForDatasource(datasource); segments.forEach(segment -> { - locks.forEach(lock -> { if (lock.getGranularity() == LockGranularity.TIME_CHUNK) { final TimeChunkLock timeChunkLock = (TimeChunkLock) lock; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index 11696511b0c1..b7aa9e618dfb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -80,8 +80,6 @@ @Type(name = "index_realtime_appenderator", value = AppenderatorDriverRealtimeIndexTask.class), @Type(name = "noop", value = NoopTask.class), @Type(name = "compact", value = CompactionTask.class), - @Type(name = "replace", value = ReplaceTask.class), - @Type(name = "append", value = AppendTask.class) }) public interface Task { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index cc01d152de4d..6986161fffb0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -1245,17 +1245,17 @@ public Set getAllReplaceLocksForDatasource(final String datasource) { giant.lock(); try { - final NavigableMap>> dsRunning = running.get(datasource); - if (dsRunning == null) { + final NavigableMap>> activeLocks = running.get(datasource); + if (activeLocks == null) { return ImmutableSet.of(); } - return dsRunning.values() - .stream() - .flatMap(map -> map.values().stream()) - .flatMap(Collection::stream) - .map(TaskLockPosse::getTaskLock) - .filter(taskLock -> taskLock.getType().equals(TaskLockType.REPLACE)) - .collect(Collectors.toSet()); + return activeLocks.values() + .stream() + .flatMap(map -> map.values().stream()) + .flatMap(Collection::stream) + .map(TaskLockPosse::getTaskLock) + .filter(taskLock -> !taskLock.isRevoked() && taskLock.getType().equals(TaskLockType.REPLACE)) + .collect(Collectors.toSet()); } finally { giant.unlock(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java index f38dab1c2541..2960ce63436e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java @@ -20,7 +20,6 @@ package org.apache.druid.indexing.common.task; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import org.apache.druid.indexing.common.MultipleFileTaskReportFileWriter; import org.apache.druid.indexing.common.TaskStorageDirTracker; import org.apache.druid.indexing.common.TaskToolbox; @@ -30,6 +29,8 @@ import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; +import org.apache.druid.indexing.common.task.batch.parallel.AppendTask; +import org.apache.druid.indexing.common.task.batch.parallel.ReplaceTask; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.overlord.TaskQueue; import org.apache.druid.indexing.overlord.TaskRunner; @@ -103,10 +104,8 @@ public void test() throws Exception "DS", Intervals.of("2023/2024"), Granularities.YEAR, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -114,10 +113,8 @@ public void test() throws Exception "DS", Intervals.of("2023/2024"), Granularities.YEAR, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); ReplaceTask replaceTask1 = new ReplaceTask( @@ -125,10 +122,8 @@ public void test() throws Exception "DS", Intervals.of("2023/2024"), Granularities.YEAR, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); ReplaceTask replaceTask2 = new ReplaceTask( @@ -136,10 +131,8 @@ public void test() throws Exception "DS", Intervals.of("2023/2024"), Granularities.YEAR, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); // Create a set of initial segments @@ -188,10 +181,8 @@ public void testRRAA_dailyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -199,10 +190,8 @@ public void testRRAA_dailyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(replaceTask0); @@ -237,10 +226,8 @@ public void testRAAR_dailyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -248,10 +235,8 @@ public void testRAAR_dailyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(replaceTask0); @@ -287,10 +272,8 @@ public void testRARA_dailyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -298,10 +281,8 @@ public void testRARA_dailyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(replaceTask0); @@ -339,10 +320,8 @@ public void testARRA_dailyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -350,10 +329,8 @@ public void testARRA_dailyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(appendTask0); @@ -388,10 +365,8 @@ public void testARAR_dailyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -399,10 +374,8 @@ public void testARAR_dailyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(appendTask0); @@ -439,10 +412,8 @@ public void testAARR_dailyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -450,10 +421,8 @@ public void testAARR_dailyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(appendTask0); @@ -489,10 +458,8 @@ public void testRRAA_monthlyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -500,10 +467,8 @@ public void testRRAA_monthlyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(replaceTask0); @@ -538,10 +503,8 @@ public void testRAAR_monthlyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -549,10 +512,8 @@ public void testRAAR_monthlyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(replaceTask0); @@ -588,10 +549,8 @@ public void testRARA_monthlyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -599,10 +558,8 @@ public void testRARA_monthlyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(replaceTask0); @@ -640,10 +597,8 @@ public void testARRA_monthlyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -651,10 +606,8 @@ public void testARRA_monthlyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(appendTask0); @@ -689,10 +642,8 @@ public void testARAR_monthlyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -700,10 +651,8 @@ public void testARAR_monthlyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(appendTask0); @@ -740,10 +689,8 @@ public void testAARR_monthlyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -751,10 +698,8 @@ public void testAARR_monthlyReplaceDailyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(appendTask0); @@ -790,10 +735,8 @@ public void testRRAA_dailyReplaceMonthlyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -801,10 +744,8 @@ public void testRRAA_dailyReplaceMonthlyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(replaceTask0); @@ -839,10 +780,8 @@ public void testRAAR_dailyReplaceMonthlyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -850,10 +789,8 @@ public void testRAAR_dailyReplaceMonthlyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(replaceTask0); @@ -889,10 +826,8 @@ public void testRARA_dailyReplaceMonthlyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -900,10 +835,8 @@ public void testRARA_dailyReplaceMonthlyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(replaceTask0); @@ -941,10 +874,8 @@ public void testARRA_dailyReplaceMonthlyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -952,10 +883,8 @@ public void testARRA_dailyReplaceMonthlyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(appendTask0); @@ -990,10 +919,8 @@ public void testARAR_dailyReplaceMonthlyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -1001,10 +928,8 @@ public void testARAR_dailyReplaceMonthlyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(appendTask0); @@ -1041,10 +966,8 @@ public void testAARR_dailyReplaceMonthlyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -1052,10 +975,8 @@ public void testAARR_dailyReplaceMonthlyAppend() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(appendTask0); @@ -1089,10 +1010,8 @@ public void testMultipleAppend() throws Exception "DS", Intervals.of("2023/2024"), Granularities.YEAR, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -1100,10 +1019,8 @@ public void testMultipleAppend() throws Exception "DS", Intervals.of("2023/2024"), Granularities.YEAR, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); AppendTask appendTask1 = new AppendTask( @@ -1111,10 +1028,8 @@ public void testMultipleAppend() throws Exception "DS", Intervals.of("2023/2024"), Granularities.YEAR, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(appendTask0); @@ -1157,10 +1072,8 @@ public void testMultipleGranularities() throws Exception "DS", Intervals.of("2023/2024"), Granularities.YEAR, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "REPLACE", - "numCorePartitions", 1 - ) + null, + null ); AppendTask appendTask0 = new AppendTask( @@ -1168,10 +1081,8 @@ public void testMultipleGranularities() throws Exception "DS", Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); AppendTask appendTask1 = new AppendTask( @@ -1179,10 +1090,8 @@ public void testMultipleGranularities() throws Exception "DS", Intervals.of("2023-07-01/2024-01-01"), Granularities.QUARTER, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); AppendTask appendTask2 = new AppendTask( @@ -1190,10 +1099,8 @@ public void testMultipleGranularities() throws Exception "DS", Intervals.of("2023-12-01/2024-01-01"), Granularities.MONTH, - ImmutableMap.of( - Tasks.TASK_LOCK_TYPE, "APPEND", - "numPartitions", 1 - ) + null, + null ); taskQueue.add(appendTask0); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppendTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java similarity index 86% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppendTask.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java index bbcaa8ece272..1790d6737e52 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppendTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java @@ -17,10 +17,8 @@ * under the License. */ -package org.apache.druid.indexing.common.task; +package org.apache.druid.indexing.common.task.batch.parallel; -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -35,6 +33,7 @@ import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -54,7 +53,6 @@ import java.io.IOException; import java.util.Collections; import java.util.HashSet; -import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.concurrent.CountDownLatch; @@ -67,27 +65,12 @@ * Begins running by acquiring an APPEND lock and immediately allocates pending segments. * * Task ends after publishing these pending segments and relevant metadata entries in a transaction - * Replace lock exists with version V3 - * V0 -> PS0 -> append task begins - * V1 -> (S1-0) -> replace task begins and published and completed - * V2 -> (S2-0) -> replace task begins and published and completed - * V3 -> Replace acquired lock - * - * append task publishes PS0 -> S0-0, S1-1, S2-1 ; (S0-0, V3); Append task has completed - * - * - * (S3-0, 1) today - * (S3-0, 2), (S3-1, 2) needs to happen - * V3 replace task finishes -> (S3-0, S0-0 == S3-1) - * - * segment metadata -> Publish all pending segments and also create copies for greater versions for which used segments exist - * forward metadata -> Publish mapping of the original segments to the EXCLUSIVE lock held for the same interval when present */ public class AppendTask extends AbstractTask { private final Interval interval; private final Granularity segmentGranularity; - private final String lockType; + private final TaskLockType lockType; private final int priority; private final int numPartitions; private final CountDownLatch readyLatch = new CountDownLatch(1); @@ -102,24 +85,25 @@ public class AppendTask extends AbstractTask private final AtomicInteger sequenceId = new AtomicInteger(0); public AppendTask( - @JsonProperty("id") String id, - @JsonProperty("dataSource") String dataSource, - @JsonProperty("interval") Interval interval, - @JsonProperty("segmentGranularity") Granularity segmentGranularity, - @JsonProperty("context") Map context + String id, + String dataSource, + Interval interval, + Granularity segmentGranularity, + Integer priority, + Integer numPartitions ) { super( id == null ? StringUtils.format("replace_%s_%s", DateTimes.nowUtc(), UUID.randomUUID().toString()) : id, dataSource == null ? "none" : dataSource, - context, + null, IngestionMode.APPEND ); this.interval = interval; this.segmentGranularity = segmentGranularity; - this.lockType = getContextValue(Tasks.TASK_LOCK_TYPE, "EXCLUSIVE"); - this.priority = getContextValue(Tasks.PRIORITY_KEY, 0); - this.numPartitions = getContextValue("numPartitions", 0); + this.lockType = TaskLockType.APPEND; + this.priority = priority == null ? 50 : priority; + this.numPartitions = numPartitions == null ? 1 : numPartitions; } @Override @@ -129,7 +113,6 @@ public String getType() } @Nonnull - @JsonIgnore @Override public Set getInputSourceResources() { @@ -143,7 +126,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception return tryTimeChunkLockSingleInterval( new SurrogateTaskActionClient(getId(), taskActionClient), interval, - TaskLockType.valueOf(lockType) + lockType ); } @@ -219,7 +202,7 @@ private SegmentIdWithShardSpec allocateNewSegmentForDate(DateTime time) false, NumberedPartialShardSpec.instance(), LockGranularity.TIME_CHUNK, - TaskLockType.valueOf(lockType) + lockType ); final SegmentIdWithShardSpec id = toolbox.getTaskActionClient().submit(allocateAction); pendingSegments.add(id); @@ -249,7 +232,7 @@ private Set allocatePendingSegments(TaskToolbox toolbox) false, NumberedPartialShardSpec.instance(), LockGranularity.TIME_CHUNK, - TaskLockType.valueOf(lockType) + lockType ); final SegmentIdWithShardSpec id = toolbox.getTaskActionClient().submit(allocateAction); pendingSegments.add( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ReplaceTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java similarity index 90% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/ReplaceTask.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java index 5bcfb4ee58e6..9e09343679b5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ReplaceTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java @@ -17,11 +17,8 @@ * under the License. */ -package org.apache.druid.indexing.common.task; +package org.apache.druid.indexing.common.task.batch.parallel; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -35,6 +32,7 @@ import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; @@ -52,7 +50,6 @@ import java.io.IOException; import java.util.Collections; import java.util.HashSet; -import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.concurrent.CountDownLatch; @@ -70,7 +67,7 @@ public class ReplaceTask extends AbstractTask { private final Interval interval; private final Granularity segmentGranularity; - private final String lockType; + private final TaskLockType lockType; private final int priority; private final int numCorePartitions; private final CountDownLatch readyLatch = new CountDownLatch(1); @@ -79,26 +76,26 @@ public class ReplaceTask extends AbstractTask private final CountDownLatch runComplete = new CountDownLatch(1); private String version; - @JsonCreator public ReplaceTask( - @JsonProperty("id") String id, - @JsonProperty("dataSource") String dataSource, - @JsonProperty("interval") Interval interval, - @JsonProperty("segmentGranularity") Granularity segmentGranularity, - @JsonProperty("context") Map context + String id, + String dataSource, + Interval interval, + Granularity segmentGranularity, + Integer priority, + Integer numCorePartitions ) { super( id == null ? StringUtils.format("replace_%s_%s", DateTimes.nowUtc(), UUID.randomUUID().toString()) : id, dataSource == null ? "none" : dataSource, - context, + null, IngestionMode.REPLACE ); this.interval = interval; this.segmentGranularity = segmentGranularity; - this.lockType = getContextValue(Tasks.TASK_LOCK_TYPE, "EXCLUSIVE"); - this.priority = getContextValue(Tasks.PRIORITY_KEY, 0); - this.numCorePartitions = getContextValue("numCorePartitions", 0); + this.lockType = TaskLockType.REPLACE; + this.priority = priority == null ? 50 : priority; + this.numCorePartitions = numCorePartitions == null ? 1 : numCorePartitions; } @Override @@ -108,7 +105,6 @@ public String getType() } @Nonnull - @JsonIgnore @Override public Set getInputSourceResources() { @@ -131,7 +127,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception return tryTimeChunkLockSingleInterval( new SurrogateTaskActionClient(getId(), taskActionClient), interval, - TaskLockType.valueOf(lockType) + lockType ); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java index 7df4840de306..a71e4cdb6d36 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java @@ -72,9 +72,4 @@ default boolean supportsEmptyPublish() { return false; } - - - // append starts (2023-01-01/2023-02-01) - V0-0 pending segment - // replace starts and ends (2023-01-01/2024-01-01) - V1-0 committed segment - // append ends (2023-01-01/2023-02-01) - V0-0 committed segment, (2023-01-01/2024-01-01) V1-1 committed segment } From 6a9e6e78f2bba2bc1219f00c68d49a570d18c874 Mon Sep 17 00:00:00 2001 From: Amatya Date: Mon, 7 Aug 2023 10:56:46 +0530 Subject: [PATCH 05/43] Address preliminary feedback --- .../MaterializedViewSupervisorTest.java | 6 +- .../DatasourceOptimizerTest.java | 4 +- .../common/actions/CommitMetadataAction.java | 127 ++++ .../common/actions/SegmentInsertAction.java | 2 +- .../actions/SegmentMetadataUpdateAction.java | 2 +- .../common/actions/SegmentNukeAction.java | 2 +- .../SegmentTransactionalAppendAction.java | 42 +- .../SegmentTransactionalInsertAction.java | 4 +- .../SegmentTransactionalReplaceAction.java | 72 +-- .../indexing/common/actions/TaskAction.java | 1 + .../common/task/AbstractBatchIndexTask.java | 32 +- .../druid/indexing/common/task/IndexTask.java | 13 +- .../indexing/common/task/TaskLockHelper.java | 41 +- .../druid/indexing/common/task/Tasks.java | 3 + .../parallel/ParallelIndexSupervisorTask.java | 10 +- .../SinglePhaseParallelIndexTaskRunner.java | 19 +- .../batch/parallel/SinglePhaseSubTask.java | 3 +- .../druid/indexing/overlord/TaskLockbox.java | 4 +- .../overlord/TimeChunkLockRequest.java | 13 +- .../SeekableStreamIndexTask.java | 2 +- .../actions/RetrieveSegmentsActionsTest.java | 4 +- .../actions/SegmentAllocateActionTest.java | 12 +- .../actions/SegmentInsertActionTest.java | 4 +- ...penderatorDriverRealtimeIndexTaskTest.java | 8 +- .../task/KillUnusedSegmentsTaskTest.java | 8 +- .../task/batch/parallel/AppendTask.java | 2 +- .../task/batch/parallel/ReplaceTask.java | 2 +- .../indexing/overlord/TaskLifecycleTest.java | 4 +- .../overlord/TaskLockBoxConcurrencyTest.java | 9 +- .../indexing/overlord/TaskLockboxTest.java | 10 +- ...TestIndexerMetadataStorageCoordinator.java | 24 +- .../indexing/overlord/DataSourceMetadata.java | 4 +- .../IndexerMetadataStorageCoordinator.java | 26 +- .../IndexerSQLMetadataStorageCoordinator.java | 611 ++++++++++++++++-- ...exerSQLMetadataStorageCoordinatorTest.java | 112 ++-- 35 files changed, 896 insertions(+), 346 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitMetadataAction.java diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java index d822948881b3..98b0ed013d16 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java @@ -192,8 +192,8 @@ public void testCheckSegments() throws IOException 1024 ) ); - indexerMetadataStorageCoordinator.announceHistoricalSegments(baseSegments); - indexerMetadataStorageCoordinator.announceHistoricalSegments(derivativeSegments); + indexerMetadataStorageCoordinator.commitSegments(baseSegments); + indexerMetadataStorageCoordinator.commitSegments(derivativeSegments); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); @@ -252,7 +252,7 @@ public void testCheckSegmentsAndSubmitTasks() throws IOException 1024 ) ); - indexerMetadataStorageCoordinator.announceHistoricalSegments(baseSegments); + indexerMetadataStorageCoordinator.commitSegments(baseSegments); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); diff --git a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java index 520aa3193b63..673c2531999c 100644 --- a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java +++ b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java @@ -160,7 +160,7 @@ public void testOptimize() throws InterruptedException 1024 * 1024 ); try { - metadataStorageCoordinator.announceHistoricalSegments(Sets.newHashSet(segment)); + metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment)); announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper); } catch (IOException e) { @@ -185,7 +185,7 @@ public void testOptimize() throws InterruptedException 1024 ); try { - metadataStorageCoordinator.announceHistoricalSegments(Sets.newHashSet(segment)); + metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment)); announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper); } catch (IOException e) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitMetadataAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitMetadataAction.java new file mode 100644 index 000000000000..608a424b5979 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitMetadataAction.java @@ -0,0 +1,127 @@ +/* + * 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.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.overlord.SegmentPublishResult; + +import javax.annotation.Nullable; + +/** + * A stream ingestion task didn't ingest any rows and created no segments (e.g., all records were unparseable), + * but still needs to update metadata with the progress that the task made. + */ +public class CommitMetadataAction implements TaskAction +{ + @Nullable + private final DataSourceMetadata startMetadata; + @Nullable + private final DataSourceMetadata endMetadata; + @Nullable + private final String dataSource; + + public static CommitMetadataAction create( + String dataSource, + DataSourceMetadata startMetadata, + DataSourceMetadata endMetadata + ) + { + return new CommitMetadataAction(startMetadata, endMetadata, dataSource); + } + + @JsonCreator + private CommitMetadataAction( + @JsonProperty("startMetadata") @Nullable DataSourceMetadata startMetadata, + @JsonProperty("endMetadata") @Nullable DataSourceMetadata endMetadata, + @JsonProperty("dataSource") @Nullable String dataSource + ) + { + this.startMetadata = startMetadata; + this.endMetadata = endMetadata; + this.dataSource = dataSource; + } + + @JsonProperty + @Nullable + public DataSourceMetadata getStartMetadata() + { + return startMetadata; + } + + @JsonProperty + @Nullable + public DataSourceMetadata getEndMetadata() + { + return endMetadata; + } + + @JsonProperty + @Nullable + public String getDataSource() + { + return dataSource; + } + + @Override + public TypeReference getReturnTypeReference() + { + return new TypeReference() + { + }; + } + + /** + * Performs some sanity checks and publishes the given segments. + */ + @Override + public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) + { + try { + return toolbox.getIndexerMetadataStorageCoordinator().commitMetadataOnly( + dataSource, + startMetadata, + endMetadata + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public boolean isAudited() + { + return true; + } + + @Override + public String toString() + { + return "CommitMetadataAction{" + + "startMetadata=" + startMetadata + + ", endMetadata=" + endMetadata + + ", dataSource='" + dataSource + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java index b81d3d377047..9b7f964da4ff 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java @@ -62,7 +62,7 @@ public TypeReference> getReturnTypeReference() /** * Behaves similarly to - * {@link org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator#announceHistoricalSegments}, + * {@link org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator#commitSegments}, * with startMetadata and endMetadata both null. */ @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java index 143805c82d4b..33d79f003a72 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java @@ -69,7 +69,7 @@ public Void perform(Task task, TaskActionToolbox toolbox) try { toolbox.getTaskLockbox().doInCriticalSection( task, - segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()), + segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), CriticalAction.builder() .onValidLocks( () -> { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java index 562e8681c761..484e715d0020 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java @@ -71,7 +71,7 @@ public Void perform(Task task, TaskActionToolbox toolbox) try { toolbox.getTaskLockbox().doInCriticalSection( task, - segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()), + segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), CriticalAction.builder() .onValidLocks( () -> { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java index bc202a1a0fe1..e0d155408dc0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java @@ -55,30 +55,26 @@ public class SegmentTransactionalAppendAction implements TaskAction segments, @Nullable DataSourceMetadata startMetadata, @Nullable DataSourceMetadata endMetadata ) { - return new SegmentTransactionalAppendAction(segments, startMetadata, endMetadata, null); + return new SegmentTransactionalAppendAction(segments, startMetadata, endMetadata); } @JsonCreator private SegmentTransactionalAppendAction( @JsonProperty("segments") @Nullable Set segments, @JsonProperty("startMetadata") @Nullable DataSourceMetadata startMetadata, - @JsonProperty("endMetadata") @Nullable DataSourceMetadata endMetadata, - @JsonProperty("dataSource") @Nullable String dataSource + @JsonProperty("endMetadata") @Nullable DataSourceMetadata endMetadata ) { this.segments = segments == null ? ImmutableSet.of() : ImmutableSet.copyOf(segments); this.startMetadata = startMetadata; this.endMetadata = endMetadata; - this.dataSource = dataSource; } @JsonProperty @@ -101,13 +97,6 @@ public DataSourceMetadata getEndMetadata() return endMetadata; } - @JsonProperty - @Nullable - public String getDataSource() - { - return dataSource; - } - @Override public TypeReference getReturnTypeReference() { @@ -124,22 +113,6 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) { final SegmentPublishResult retVal; - if (segments.isEmpty()) { - // A stream ingestion task didn't ingest any rows and created no segments (e.g., all records were unparseable), - // but still needs to update metadata with the progress that the task made. - try { - retVal = toolbox.getIndexerMetadataStorageCoordinator().commitMetadataOnly( - dataSource, - startMetadata, - endMetadata - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - return retVal; - } - final Set allSegments = new HashSet<>(segments); String datasource = task.getDataSource(); @@ -165,17 +138,15 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) try { retVal = toolbox.getTaskLockbox().doInCriticalSection( task, - allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toList()), + allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), CriticalAction.builder() .onValidLocks( - () -> toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments( + () -> toolbox.getIndexerMetadataStorageCoordinator().commitAppendSegments( segments, - null, startMetadata, endMetadata, appendSegmentLockMap, - taskLockInfos, - true + taskLockInfos ) ) .onInvalidLocks( @@ -233,7 +204,6 @@ public String toString() "segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + ", startMetadata=" + startMetadata + ", endMetadata=" + endMetadata + - ", dataSource='" + dataSource + '\'' + '}'; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java index a0567dce04bf..5acc048fdfe0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java @@ -219,10 +219,10 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) try { retVal = toolbox.getTaskLockbox().doInCriticalSection( task, - allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toList()), + allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), CriticalAction.builder() .onValidLocks( - () -> toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments( + () -> toolbox.getIndexerMetadataStorageCoordinator().commitSegments( segments, segmentsToBeDropped, startMetadata, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index 23c8fae0aa2e..4dd3ea7a389a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -27,7 +27,6 @@ import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; -import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.TaskLockInfo; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; @@ -65,38 +64,25 @@ public class SegmentTransactionalReplaceAction implements TaskAction segmentsToBeDropped; - @Nullable - private final DataSourceMetadata startMetadata; - @Nullable - private final DataSourceMetadata endMetadata; - @Nullable - private final String dataSource; - - public static SegmentTransactionalReplaceAction overwriteAction( + public static SegmentTransactionalReplaceAction create( @Nullable Set segmentsToBeOverwritten, @Nullable Set segmentsToBeDropped, Set segmentsToPublish ) { - return new SegmentTransactionalReplaceAction(segmentsToBeOverwritten, segmentsToBeDropped, segmentsToPublish, null, null, null); + return new SegmentTransactionalReplaceAction(segmentsToBeOverwritten, segmentsToBeDropped, segmentsToPublish); } @JsonCreator private SegmentTransactionalReplaceAction( @JsonProperty("segmentsToBeOverwritten") @Nullable Set segmentsToBeOverwritten, @JsonProperty("segmentsToBeDropped") @Nullable Set segmentsToBeDropped, - @JsonProperty("segments") @Nullable Set segments, - @JsonProperty("startMetadata") @Nullable DataSourceMetadata startMetadata, - @JsonProperty("endMetadata") @Nullable DataSourceMetadata endMetadata, - @JsonProperty("dataSource") @Nullable String dataSource + @JsonProperty("segments") Set segments ) { this.segmentsToBeOverwritten = segmentsToBeOverwritten; this.segmentsToBeDropped = segmentsToBeDropped; - this.segments = segments == null ? ImmutableSet.of() : ImmutableSet.copyOf(segments); - this.startMetadata = startMetadata; - this.endMetadata = endMetadata; - this.dataSource = dataSource; + this.segments = ImmutableSet.copyOf(segments); } @JsonProperty @@ -119,27 +105,6 @@ public Set getSegments() return segments; } - @JsonProperty - @Nullable - public DataSourceMetadata getStartMetadata() - { - return startMetadata; - } - - @JsonProperty - @Nullable - public DataSourceMetadata getEndMetadata() - { - return endMetadata; - } - - @JsonProperty - @Nullable - public String getDataSource() - { - return dataSource; - } - @Override public TypeReference getReturnTypeReference() { @@ -156,22 +121,6 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) { final SegmentPublishResult retVal; - if (segments.isEmpty()) { - // A stream ingestion task didn't ingest any rows and created no segments (e.g., all records were unparseable), - // but still needs to update metadata with the progress that the task made. - try { - retVal = toolbox.getIndexerMetadataStorageCoordinator().commitMetadataOnly( - dataSource, - startMetadata, - endMetadata - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - return retVal; - } - final Set allSegments = new HashSet<>(segments); TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), allSegments); @@ -189,17 +138,13 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) try { retVal = toolbox.getTaskLockbox().doInCriticalSection( task, - allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toList()), + allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), CriticalAction.builder() .onValidLocks( - () -> toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments( + () -> toolbox.getIndexerMetadataStorageCoordinator().commitReplaceSegments( segments, segmentsToBeDropped, - startMetadata, - endMetadata, - null, - taskLockInfos, - false + taskLockInfos ) ) .onInvalidLocks( @@ -255,9 +200,6 @@ public String toString() return "SegmentTransactionalInsertAction{" + "segmentsToBeOverwritten=" + SegmentUtils.commaSeparatedIdentifiers(segmentsToBeOverwritten) + ", segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + - ", startMetadata=" + startMetadata + - ", endMetadata=" + endMetadata + - ", dataSource='" + dataSource + '\'' + ", segmentsToBeDropped=" + SegmentUtils.commaSeparatedIdentifiers(segmentsToBeDropped) + '}'; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java index 171d53b9cdd6..82833c5ea64f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java @@ -36,6 +36,7 @@ @JsonSubTypes.Type(name = "lockRelease", value = LockReleaseAction.class), @JsonSubTypes.Type(name = "segmentInsertion", value = SegmentInsertAction.class), @JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class), + @JsonSubTypes.Type(name = "commitMetadata", value = SegmentTransactionalInsertAction.class), @JsonSubTypes.Type(name = "segmentTransactionalAppend", value = SegmentTransactionalAppendAction.class), @JsonSubTypes.Type(name = "segmentTransactionalReplace", value = SegmentTransactionalReplaceAction.class), // Type name doesn't correspond to the name of the class for backward compatibility. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index b6e9cdb9f9e7..5d08d8e531b6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -39,7 +39,6 @@ import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.indexing.common.task.batch.MaxAllowedLocksExceededException; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; @@ -289,22 +288,21 @@ public TaskLockHelper getTaskLockHelper() * * @return whether the lock was acquired */ - public boolean determineLockGranularityAndTryLock(TaskActionClient client, List intervals, IndexIOConfig ioConfig) + public boolean determineLockGranularityAndTryLock(TaskActionClient client, List intervals) throws IOException { final boolean forceTimeChunkLock = getContextValue( Tasks.FORCE_TIME_CHUNK_LOCK_KEY, Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK ); - IngestionMode ingestionMode = getIngestionMode(); - final TaskLockType taskLockType = TaskLockType.valueOf(getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name())); + final IngestionMode ingestionMode = getIngestionMode(); // Respect task context value most. if (forceTimeChunkLock || ingestionMode == IngestionMode.REPLACE) { log.info( "forceTimeChunkLock[%s] is set to true or mode[%s] is replace. Use timeChunk lock", forceTimeChunkLock, ingestionMode ); - taskLockHelper = new TaskLockHelper(false, taskLockType); + taskLockHelper = new TaskLockHelper(false, getContext(), ingestionMode); if (!intervals.isEmpty()) { return tryTimeChunkLock(client, intervals); } else { @@ -313,7 +311,11 @@ public boolean determineLockGranularityAndTryLock(TaskActionClient client, List< } else { if (!intervals.isEmpty()) { final LockGranularityDetermineResult result = determineSegmentGranularity(client, intervals); - taskLockHelper = new TaskLockHelper(result.lockGranularity == LockGranularity.SEGMENT, taskLockType); + taskLockHelper = new TaskLockHelper( + result.lockGranularity == LockGranularity.SEGMENT, + getContext(), + ingestionMode + ); return tryLockWithDetermineResult(client, result); } else { // This branch is the only one that will not initialize taskLockHelper. @@ -341,11 +343,11 @@ boolean determineLockGranularityAndTryLockWithSegments( Tasks.FORCE_TIME_CHUNK_LOCK_KEY, Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK ); - final TaskLockType taskLockType = TaskLockType.valueOf(getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name())); + final IngestionMode ingestionMode = getIngestionMode(); if (forceTimeChunkLock) { log.info("[%s] is set to true in task context. Use timeChunk lock", Tasks.FORCE_TIME_CHUNK_LOCK_KEY); - taskLockHelper = new TaskLockHelper(false, taskLockType); + taskLockHelper = new TaskLockHelper(false, getContext(), ingestionMode); segmentCheckFunction.accept(LockGranularity.TIME_CHUNK, segments); return tryTimeChunkLock( client, @@ -353,7 +355,11 @@ boolean determineLockGranularityAndTryLockWithSegments( ); } else { final LockGranularityDetermineResult result = determineSegmentGranularity(segments); - taskLockHelper = new TaskLockHelper(result.lockGranularity == LockGranularity.SEGMENT, taskLockType); + taskLockHelper = new TaskLockHelper( + result.lockGranularity == LockGranularity.SEGMENT, + getContext(), + ingestionMode + ); segmentCheckFunction.accept(result.lockGranularity, segments); return tryLockWithDetermineResult(client, result); } @@ -429,7 +435,7 @@ protected boolean tryTimeChunkLock(TaskActionClient client, List inter } prev = cur; - final TaskLockType taskLockType = TaskLockType.valueOf(getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name())); + final TaskLockType taskLockType = TaskLockHelper.lockTypeFrom(false, getContext(), getIngestionMode()); final TaskLock lock = client.submit(new TimeChunkLockTryAcquireAction(taskLockType, cur)); if (lock == null) { return false; @@ -726,9 +732,6 @@ public static NonnullPair findIntervalAndVersion( } } // We don't have a lock for this interval, so we should lock it now. - if (taskLockType == null) { - taskLockType = TaskLockType.EXCLUSIVE; - } final TaskLock lock = Preconditions.checkNotNull( toolbox.getTaskActionClient().submit( new TimeChunkLockTryAcquireAction(taskLockType, interval) @@ -787,8 +790,7 @@ Pair lookupVersion(DateTime timestamp) protected SegmentIdWithShardSpec allocateNewSegmentForTombstone( IngestionSpec ingestionSchema, - DateTime timestamp, - TaskToolbox toolbox + DateTime timestamp ) { // Since tombstones are derived from inputIntervals, inputIntervals cannot be empty for replace, and locks are diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index f061e121e06d..ebc728f9d87a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -256,8 +256,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception } return determineLockGranularityAndTryLock( taskActionClient, - ingestionSchema.dataSchema.getGranularitySpec().inputIntervals(), - ingestionSchema.getIOConfig() + ingestionSchema.dataSchema.getGranularitySpec().inputIntervals() ); } @@ -527,7 +526,8 @@ public TaskStatus runTask(final TaskToolbox toolbox) final DataSchema dataSchema; if (determineIntervals) { final boolean gotLocks = determineLockGranularityAndTryLock( - toolbox.getTaskActionClient(), allocateIntervals, ingestionSchema.getIOConfig() + toolbox.getTaskActionClient(), + allocateIntervals ); if (!gotLocks) { throw new ISE("Failed to get locks for intervals[%s]", allocateIntervals); @@ -920,11 +920,11 @@ private TaskStatus generateAndPublishSegments( switch (lockType) { case REPLACE: return toolbox.getTaskActionClient().submit( - SegmentTransactionalReplaceAction.overwriteAction(segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish) + SegmentTransactionalReplaceAction.create(segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish) ); case APPEND: return toolbox.getTaskActionClient().submit( - SegmentTransactionalAppendAction.appendAction(segmentsToPublish, null, null) + SegmentTransactionalAppendAction.create(segmentsToPublish, null, null) ); default: return toolbox.getTaskActionClient().submit( @@ -997,8 +997,7 @@ private TaskStatus generateAndPublishSegments( for (Interval interval : tombstoneIntervals) { SegmentIdWithShardSpec segmentIdWithShardSpec = allocateNewSegmentForTombstone( ingestionSchema, - interval.getStart(), - toolbox + interval.getStart() ); tombstonesAndVersions.put(interval, segmentIdWithShardSpec); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskLockHelper.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskLockHelper.java index 1584c88f4df1..77e5b66f969d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskLockHelper.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskLockHelper.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.common.task; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableSet; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.actions.SegmentLockTryAcquireAction; @@ -91,10 +92,10 @@ public short getMinorVersionForNewSegments() } } - public TaskLockHelper(boolean useSegmentLock, TaskLockType taskLockType) + public TaskLockHelper(boolean useSegmentLock, Map context, AbstractTask.IngestionMode mode) { this.useSegmentLock = useSegmentLock; - this.taskLockType = taskLockType; + this.taskLockType = lockTypeFrom(useSegmentLock, context, mode); } public boolean isUseSegmentLock() @@ -109,12 +110,7 @@ public LockGranularity getLockGranularityToUse() public TaskLockType getLockTypeToUse() { - return taskLockType == null ? TaskLockType.EXCLUSIVE : taskLockType; - } - - public boolean hasLockedExistingSegments() - { - return !lockedExistingSegments.isEmpty(); + return taskLockType; } public boolean hasOverwritingRootGenerationPartition(Interval interval) @@ -132,6 +128,9 @@ public OverwritingRootGenerationPartitions getOverwritingRootGenerationPartition return overwritingRootGenPartitions.get(interval); } + /** + * Verify and lock existing segments when using a SegmentLock + */ boolean verifyAndLockExistingSegments(TaskActionClient actionClient, List segments) throws IOException { @@ -312,4 +311,30 @@ public static void verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(List< ); } } + + public static TaskLockType lockTypeFrom( + boolean useSegmentLock, + Map context, + AbstractTask.IngestionMode mode + ) + { + if (useSegmentLock) { + return Tasks.DEFAULT_TASK_LOCK_TYPE; + } + + final TaskLockType lockTypeVal; + final String lockTypeName = context.get(Tasks.TASK_LOCK_TYPE).toString(); + if (lockTypeName == null) { + lockTypeVal = (boolean) context.get(Tasks.USE_SHARED_LOCK) ? TaskLockType.SHARED : TaskLockType.EXCLUSIVE; + } else { + lockTypeVal = TaskLockType.valueOf(lockTypeName); + } + + final Set appendModeCompatible = ImmutableSet.of(TaskLockType.SHARED, TaskLockType.APPEND); + if (!mode.equals(AbstractTask.IngestionMode.APPEND) && appendModeCompatible.contains(lockTypeVal)) { + return Tasks.DEFAULT_TASK_LOCK_TYPE; + } else { + return lockTypeVal; + } + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java index f3f391d4f35e..0709e6088095 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.common.task; import org.apache.curator.shaded.com.google.common.base.Verify; +import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -47,11 +48,13 @@ public class Tasks public static final boolean DEFAULT_FORCE_TIME_CHUNK_LOCK = true; public static final boolean DEFAULT_STORE_COMPACTION_STATE = false; public static final boolean DEFAULT_USE_MAX_MEMORY_ESTIMATES = false; + public static final TaskLockType DEFAULT_TASK_LOCK_TYPE = TaskLockType.EXCLUSIVE; public static final String PRIORITY_KEY = "priority"; public static final String LOCK_TIMEOUT_KEY = "taskLockTimeout"; public static final String FORCE_TIME_CHUNK_LOCK_KEY = "forceTimeChunkLock"; public static final String STORE_EMPTY_COLUMNS_KEY = "storeEmptyColumns"; + public static final String USE_SHARED_LOCK = "useSharedLock"; public static final String TASK_LOCK_TYPE = "taskLockType"; /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 74900da9c122..8d170cb62372 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -436,8 +436,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception { return determineLockGranularityAndTryLock( taskActionClient, - ingestionSchema.getDataSchema().getGranularitySpec().inputIntervals(), - ingestionSchema.getIOConfig() + ingestionSchema.getDataSchema().getGranularitySpec().inputIntervals() ); } @@ -1159,8 +1158,7 @@ private void publishSegments( for (Interval interval : tombstoneIntervals) { SegmentIdWithShardSpec segmentIdWithShardSpec = allocateNewSegmentForTombstone( ingestionSchema, - interval.getStart(), - toolbox + interval.getStart() ); tombstonesAnShards.put(interval, segmentIdWithShardSpec); } @@ -1178,11 +1176,11 @@ private void publishSegments( switch (lockType) { case REPLACE: return toolbox.getTaskActionClient().submit( - SegmentTransactionalReplaceAction.overwriteAction(segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish) + SegmentTransactionalReplaceAction.create(segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish) ); case APPEND: return toolbox.getTaskActionClient().submit( - SegmentTransactionalAppendAction.appendAction(segmentsToPublish, null, null) + SegmentTransactionalAppendAction.create(segmentsToPublish, null, null) ); default: return toolbox.getTaskActionClient().submit( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java index 90709c79596f..77b96ca6b216 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java @@ -29,8 +29,9 @@ import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; +import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.indexing.common.task.TaskLockHelper; import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskCompleteEvent; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.NonnullPair; @@ -205,7 +206,10 @@ SubTaskSpec newTaskSpec(InputSplit split) @Deprecated public SegmentIdWithShardSpec allocateNewSegment(String dataSource, DateTime timestamp) throws IOException { - NonnullPair intervalAndVersion = findIntervalAndVersion(timestamp); + NonnullPair intervalAndVersion = findIntervalAndVersion( + timestamp, + TaskLockHelper.lockTypeFrom(false, getContext(), AbstractTask.IngestionMode.APPEND) + ); final int partitionNum = Counters.getAndIncrementInt(partitionNumCountersPerInterval, intervalAndVersion.lhs); return new SegmentIdWithShardSpec( @@ -235,7 +239,10 @@ public SegmentIdWithShardSpec allocateNewSegment( @Nullable String prevSegmentId ) throws IOException { - NonnullPair intervalAndVersion = findIntervalAndVersion(timestamp); + NonnullPair intervalAndVersion = findIntervalAndVersion( + timestamp, + TaskLockHelper.lockTypeFrom(false, getContext(), AbstractTask.IngestionMode.APPEND) + ); MutableObject segmentIdHolder = new MutableObject<>(); sequenceToSegmentIds.compute(sequenceName, (k, v) -> { @@ -283,12 +290,8 @@ public SegmentIdWithShardSpec allocateNewSegment( return segmentIdHolder.getValue(); } - NonnullPair findIntervalAndVersion(DateTime timestamp) throws IOException + NonnullPair findIntervalAndVersion(DateTime timestamp, TaskLockType taskLockType) throws IOException { - TaskLockType taskLockType = null; - if (getContext().containsKey(Tasks.TASK_LOCK_TYPE)) { - taskLockType = TaskLockType.valueOf(getContext().get(Tasks.TASK_LOCK_TYPE).toString()); - } return AbstractBatchIndexTask.findIntervalAndVersion(getToolbox(), ingestionSchema, timestamp, taskLockType); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index 349e9974fb7c..183cde7c66d0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -218,8 +218,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws IOException { return determineLockGranularityAndTryLock( new SurrogateTaskActionClient(supervisorTaskId, taskActionClient), - ingestionSchema.getDataSchema().getGranularitySpec().inputIntervals(), - ingestionSchema.getIOConfig() + ingestionSchema.getDataSchema().getGranularitySpec().inputIntervals() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 6986161fffb0..54666ed8630c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -769,7 +769,7 @@ private SegmentIdWithShardSpec allocateSegmentId(LockRequestForNewSegment reques * @param intervals intervals * @param action action to be performed inside of the critical section */ - public T doInCriticalSection(Task task, List intervals, CriticalAction action) throws Exception + public T doInCriticalSection(Task task, Set intervals, CriticalAction action) throws Exception { giant.lock(); @@ -786,7 +786,7 @@ public T doInCriticalSection(Task task, List intervals, CriticalAc * It doesn't check other semantics like acquired locks are enough to overwrite existing segments. * This kind of semantic should be checked in each caller of {@link #doInCriticalSection}. */ - private boolean isTaskLocksValid(Task task, List intervals) + private boolean isTaskLocksValid(Task task, Set intervals) { giant.lock(); try { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java index 3a550b4a0a03..b99d14a8e4fe 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java @@ -24,13 +24,13 @@ import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TimeChunkLock; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.java.util.common.DateTimes; import org.joda.time.Interval; import javax.annotation.Nullable; public class TimeChunkLockRequest implements LockRequest { + private static final String MIN_VERSION = "1970-01-01T00:00:00.000Z"; private final TaskLockType lockType; private final String groupId; private final String dataSource; @@ -77,7 +77,11 @@ public TimeChunkLockRequest( this.groupId = groupId; this.dataSource = dataSource; this.interval = interval; - this.preferredVersion = preferredVersion; + if (lockType.equals(TaskLockType.APPEND) && preferredVersion == null) { + this.preferredVersion = MIN_VERSION; + } else { + this.preferredVersion = preferredVersion; + } this.priority = priority; this.revoked = revoked; } @@ -115,10 +119,7 @@ public Interval getInterval() @Override public String getVersion() { - if (lockType.equals(TaskLockType.APPEND) && preferredVersion == null) { - return "1970-01-01T00:00:00.000Z"; - } - return preferredVersion == null ? DateTimes.nowUtc().toString() : preferredVersion; + return preferredVersion; } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index b3676051b090..3aca46fbfaea 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -106,7 +106,7 @@ public SeekableStreamIndexTask( this.lockGranularityToUse = getContextValue(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK) ? LockGranularity.TIME_CHUNK : LockGranularity.SEGMENT; - this.lockTypeToUse = getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE); + this.lockTypeToUse = getContextValue(Tasks.USE_SHARED_LOCK, false) ? TaskLockType.SHARED : TaskLockType.EXCLUSIVE; } protected static String getFormattedGroupId(String dataSource, String type) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java index 6149208fc391..7d54669bbd72 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java @@ -60,7 +60,7 @@ public void setup() throws IOException expectedUnusedSegments.add(createSegment(Intervals.of("2017-10-07/2017-10-08"), "1")); actionTestKit.getMetadataStorageCoordinator() - .announceHistoricalSegments(expectedUnusedSegments); + .commitSegments(expectedUnusedSegments); expectedUnusedSegments.forEach(s -> actionTestKit.getTaskLockbox().unlock(task, s.getInterval())); @@ -70,7 +70,7 @@ public void setup() throws IOException expectedUsedSegments.add(createSegment(Intervals.of("2017-10-07/2017-10-08"), "2")); actionTestKit.getMetadataStorageCoordinator() - .announceHistoricalSegments(expectedUsedSegments); + .commitSegments(expectedUsedSegments); expectedUsedSegments.forEach(s -> actionTestKit.getTaskLockbox().unlock(task, s.getInterval())); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index b498834fbaba..13c499e47e2a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -574,7 +574,7 @@ public void testAddToExistingLinearShardSpecsSameGranularity() throws Exception { final Task task = NoopTask.create(); - taskActionTestKit.getMetadataStorageCoordinator().announceHistoricalSegments( + taskActionTestKit.getMetadataStorageCoordinator().commitSegments( ImmutableSet.of( DataSegment.builder() .dataSource(DATA_SOURCE) @@ -639,7 +639,7 @@ public void testAddToExistingNumberedShardSpecsSameGranularity() throws Exceptio { final Task task = NoopTask.create(); - taskActionTestKit.getMetadataStorageCoordinator().announceHistoricalSegments( + taskActionTestKit.getMetadataStorageCoordinator().commitSegments( ImmutableSet.of( DataSegment.builder() .dataSource(DATA_SOURCE) @@ -702,7 +702,7 @@ public void testAddToExistingNumberedShardSpecsCoarserPreferredGranularity() thr { final Task task = NoopTask.create(); - taskActionTestKit.getMetadataStorageCoordinator().announceHistoricalSegments( + taskActionTestKit.getMetadataStorageCoordinator().commitSegments( ImmutableSet.of( DataSegment.builder() .dataSource(DATA_SOURCE) @@ -741,7 +741,7 @@ public void testAddToExistingNumberedShardSpecsFinerPreferredGranularity() throw { final Task task = NoopTask.create(); - taskActionTestKit.getMetadataStorageCoordinator().announceHistoricalSegments( + taskActionTestKit.getMetadataStorageCoordinator().commitSegments( ImmutableSet.of( DataSegment.builder() .dataSource(DATA_SOURCE) @@ -780,7 +780,7 @@ public void testCannotAddToExistingNumberedShardSpecsWithCoarserQueryGranularity { final Task task = NoopTask.create(); - taskActionTestKit.getMetadataStorageCoordinator().announceHistoricalSegments( + taskActionTestKit.getMetadataStorageCoordinator().commitSegments( ImmutableSet.of( DataSegment.builder() .dataSource(DATA_SOURCE) @@ -825,7 +825,7 @@ public void testWithPartialShardSpecAndOvershadowingSegments() throws IOExceptio final ObjectMapper objectMapper = new DefaultObjectMapper(); - taskActionTestKit.getMetadataStorageCoordinator().announceHistoricalSegments( + taskActionTestKit.getMetadataStorageCoordinator().commitSegments( ImmutableSet.of( DataSegment.builder() .dataSource(DATA_SOURCE) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java index c10df89ad986..8ac5c6b517f9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java @@ -107,7 +107,7 @@ public void testSimple() throws Exception acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); actionTestKit.getTaskLockbox().doInCriticalSection( task, - Collections.singletonList(INTERVAL), + Collections.singleton(INTERVAL), CriticalAction.builder() .onValidLocks(() -> action.perform(task, actionTestKit.getTaskActionToolbox())) .onInvalidLocks( @@ -137,7 +137,7 @@ public void testFailBadVersion() throws Exception thrown.expectMessage(CoreMatchers.containsString("are not covered by locks")); final Set segments = actionTestKit.getTaskLockbox().doInCriticalSection( task, - Collections.singletonList(INTERVAL), + Collections.singleton(INTERVAL), CriticalAction.>builder() .onValidLocks(() -> action.perform(task, actionTestKit.getTaskActionToolbox())) .onInvalidLocks( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 394132759cd6..db94038e07a0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -1507,9 +1507,9 @@ private void makeToolboxFactory(final File directory) ) { @Override - public Set announceHistoricalSegments(Set segments) throws IOException + public Set commitSegments(Set segments) throws IOException { - Set result = super.announceHistoricalSegments(segments); + Set result = super.commitSegments(segments); Assert.assertFalse( "Segment latch not initialized, did you forget to call expectPublishSegments?", @@ -1523,14 +1523,14 @@ public Set announceHistoricalSegments(Set segments) th } @Override - public SegmentPublishResult announceHistoricalSegments( + public SegmentPublishResult commitSegments( Set segments, Set segmentsToDrop, DataSourceMetadata startMetadata, DataSourceMetadata endMetadata ) throws IOException { - SegmentPublishResult result = super.announceHistoricalSegments(segments, segmentsToDrop, startMetadata, endMetadata, null, null, false); + SegmentPublishResult result = super.commitSegments(segments, segmentsToDrop, startMetadata, endMetadata); Assert.assertFalse( "Segment latch not initialized, did you forget to call expectPublishSegments?", diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java index f57624ed7ddf..1453cb1d0fed 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java @@ -59,7 +59,7 @@ public void testKill() throws Exception newSegment(Intervals.of("2019-03-01/2019-04-01"), version), newSegment(Intervals.of("2019-04-01/2019-05-01"), version) ); - final Set announced = getMetadataStorageCoordinator().announceHistoricalSegments(segments); + final Set announced = getMetadataStorageCoordinator().commitSegments(segments); Assert.assertEquals(segments, announced); @@ -111,7 +111,7 @@ public void testKillWithMarkUnused() throws Exception newSegment(Intervals.of("2019-03-01/2019-04-01"), version), newSegment(Intervals.of("2019-04-01/2019-05-01"), version) ); - final Set announced = getMetadataStorageCoordinator().announceHistoricalSegments(segments); + final Set announced = getMetadataStorageCoordinator().commitSegments(segments); Assert.assertEquals(segments, announced); @@ -172,7 +172,7 @@ public void testKillBatchSizeOne() throws Exception newSegment(Intervals.of("2019-03-01/2019-04-01"), version), newSegment(Intervals.of("2019-04-01/2019-05-01"), version) ); - final Set announced = getMetadataStorageCoordinator().announceHistoricalSegments(segments); + final Set announced = getMetadataStorageCoordinator().commitSegments(segments); Assert.assertEquals(segments, announced); @@ -207,7 +207,7 @@ public void testKillBatchSizeThree() throws Exception newSegment(Intervals.of("2019-03-01/2019-04-01"), version), newSegment(Intervals.of("2019-04-01/2019-05-01"), version) ); - final Set announced = getMetadataStorageCoordinator().announceHistoricalSegments(segments); + final Set announced = getMetadataStorageCoordinator().commitSegments(segments); Assert.assertEquals(segments, announced); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java index 1790d6737e52..14a4d815fce2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java @@ -274,7 +274,7 @@ private boolean publishSegments(TaskToolbox toolbox, Set newSegment { final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish, commitMetadata) -> toolbox.getTaskActionClient().submit( - SegmentTransactionalAppendAction.appendAction(segmentsToPublish, null, null) + SegmentTransactionalAppendAction.create(segmentsToPublish, null, null) ); return publisher.publishSegments( Collections.emptySet(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java index 9e09343679b5..09835ed3a4cf 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java @@ -213,7 +213,7 @@ private boolean publishSegments(TaskToolbox toolbox, Set oldSegment { final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish, commitMetadata) -> toolbox.getTaskActionClient().submit( - SegmentTransactionalReplaceAction.overwriteAction(segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish) + SegmentTransactionalReplaceAction.create(segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish) ); return publisher.publishSegments( oldSegments, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 2ea7fcedf193..b5cfe79456a0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -563,9 +563,9 @@ private TestIndexerMetadataStorageCoordinator setUpMetadataStorageCoordinator() return new TestIndexerMetadataStorageCoordinator() { @Override - public Set announceHistoricalSegments(Set segments) + public Set commitSegments(Set segments) { - Set retVal = super.announceHistoricalSegments(segments); + Set retVal = super.commitSegments(segments); if (publishCountDown != null) { publishCountDown.countDown(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java index 9c198f429179..e482f411db29 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java @@ -42,6 +42,7 @@ import org.junit.Test; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; @@ -120,7 +121,7 @@ public void testDoInCriticalSectionWithDifferentTasks() return lockbox.doInCriticalSection( lowPriorityTask, - Collections.singletonList(interval), + Collections.singleton(interval), CriticalAction.builder() .onValidLocks( () -> { @@ -150,7 +151,7 @@ public void testDoInCriticalSectionWithDifferentTasks() return lockbox.doInCriticalSection( highPriorityTask, - Collections.singletonList(interval), + Collections.singleton(interval), CriticalAction.builder() .onValidLocks( () -> { @@ -200,7 +201,7 @@ public void testDoInCriticalSectionWithOverlappedIntervals() throws Exception final Future future1 = service.submit(() -> lockbox.doInCriticalSection( task, - intervals.subList(0, 2), + new HashSet<>(intervals.subList(0, 2)), CriticalAction.builder() .onValidLocks( () -> { @@ -223,7 +224,7 @@ public void testDoInCriticalSectionWithOverlappedIntervals() throws Exception latch.await(); return lockbox.doInCriticalSection( task, - intervals.subList(1, 3), + new HashSet<>(intervals.subList(1, 3)), CriticalAction.builder() .onValidLocks( () -> { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index c9f9d72beaae..79af0f2f2cf0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -524,7 +524,7 @@ public void testDoInCriticalSectionWithSharedLock() throws Exception Assert.assertTrue( lockbox.doInCriticalSection( task, - Collections.singletonList(interval), + Collections.singleton(interval), CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() ) ); @@ -542,7 +542,7 @@ public void testDoInCriticalSectionWithExclusiveLock() throws Exception Assert.assertTrue( lockbox.doInCriticalSection( task, - Collections.singletonList(interval), + Collections.singleton(interval), CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() ) ); @@ -561,7 +561,7 @@ public void testDoInCriticalSectionWithSmallerInterval() throws Exception Assert.assertTrue( lockbox.doInCriticalSection( task, - Collections.singletonList(interval), + Collections.singleton(interval), CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() ) ); @@ -587,7 +587,7 @@ public void testPreemptionAndDoInCriticalSection() throws Exception Assert.assertTrue( lockbox.doInCriticalSection( highPriorityTask, - Collections.singletonList(interval), + Collections.singleton(interval), CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() ) ); @@ -612,7 +612,7 @@ public void testDoInCriticalSectionWithRevokedLock() throws Exception Assert.assertFalse( lockbox.doInCriticalSection( lowPriorityTask, - Collections.singletonList(interval), + Collections.singleton(interval), CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() ) ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index 84d665b61385..23893de6251f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -118,7 +118,7 @@ public int markSegmentsAsUnusedWithinInterval(String dataSource, Interval interv } @Override - public Set announceHistoricalSegments(Set segments) + public Set commitSegments(Set segments) { Set added = new HashSet<>(); for (final DataSegment segment : segments) { @@ -141,29 +141,39 @@ public Map allocatePendingSegments } @Override - public SegmentPublishResult announceHistoricalSegments( + public SegmentPublishResult commitReplaceSegments( Set segments, Set segmentsToDrop, + @Nullable Set taskLockInfos + ) + { + // Don't actually compare metadata, just do it! + return SegmentPublishResult.ok(commitSegments(segments)); + } + + @Override + public SegmentPublishResult commitAppendSegments( + Set segments, DataSourceMetadata oldCommitMetadata, DataSourceMetadata newCommitMetadata, @Nullable Map segmentLockMap, - @Nullable Set taskLockInfos, - boolean append + @Nullable Set taskLockInfos ) { // Don't actually compare metadata, just do it! - return SegmentPublishResult.ok(announceHistoricalSegments(segments)); + return SegmentPublishResult.ok(commitSegments(segments)); } @Override - public SegmentPublishResult announceHistoricalSegments( + public SegmentPublishResult commitSegments( Set segments, Set segmentsToDrop, @Nullable DataSourceMetadata startMetadata, @Nullable DataSourceMetadata endMetadata ) { - return announceHistoricalSegments(segments, segmentsToDrop, startMetadata, endMetadata, null, null, false); + // Don't actually compare metadata, just do it! + return SegmentPublishResult.ok(commitSegments(segments)); } @Override diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java b/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java index 6cad3c12649b..a6955ece8b80 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java @@ -26,7 +26,7 @@ /** * Commit metadata for a dataSource. Used by - * {@link IndexerMetadataStorageCoordinator#announceHistoricalSegments(Set, Set, DataSourceMetadata, DataSourceMetadata)} + * {@link IndexerMetadataStorageCoordinator#commitSegments(Set, Set, DataSourceMetadata, DataSourceMetadata)} * to provide metadata transactions for segment inserts. * * Two metadata instances can be added together, and any conflicts are resolved in favor of the right-hand side. @@ -45,7 +45,7 @@ public interface DataSourceMetadata boolean isValidStart(); /** - * As in {@link IndexerMetadataStorageCoordinator#announceHistoricalSegments}, this class can represent start and + * As in {@link IndexerMetadataStorageCoordinator#commitSegments}, this class can represent start and * end of a sequence. * * This method converts itself into the one for start of a sequence. Most implementations can simply return diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index f176d1abca1f..79b5d6f68bb1 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -153,7 +153,7 @@ Collection retrieveUsedSegmentsForIntervals( * * @return set of segments actually added */ - Set announceHistoricalSegments(Set segments) throws IOException; + Set commitSegments(Set segments) throws IOException; /** * Allocates pending segments for the given requests in the pending segments table. @@ -251,10 +251,6 @@ SegmentIdWithShardSpec allocatePendingSegment( * @param endMetadata dataSource metadata post-insert will have this endMetadata merged in with * {@link DataSourceMetadata#plus(DataSourceMetadata)}. If null, this insert will not * involve a metadata transaction - * @param segmentLockMap map of segments appended with an append lock to the task lock of the exclusive lock - * held for the datasource + interval during commit - * @param taskLockInfos Set of task lock infos for which new segments with given replace the old ones - * for the datasource + interval during commit * * @return segment publish result indicating transaction success or failure, and set of segments actually published. * This method must only return a failure code if it is sure that the transaction did not happen. If it is not sure, @@ -263,21 +259,25 @@ SegmentIdWithShardSpec allocatePendingSegment( * @throws IllegalArgumentException if startMetadata and endMetadata are not either both null or both non-null * @throws RuntimeException if the state of metadata storage after this call is unknown */ - SegmentPublishResult announceHistoricalSegments( + SegmentPublishResult commitSegments( Set segments, Set segmentsToDrop, @Nullable DataSourceMetadata startMetadata, + @Nullable DataSourceMetadata endMetadata + ) throws IOException; + + SegmentPublishResult commitAppendSegments( + Set segments, + @Nullable DataSourceMetadata startMetadata, @Nullable DataSourceMetadata endMetadata, @Nullable Map segmentLockMap, - @Nullable Set taskLockInfos, - boolean append + @Nullable Set taskLockInfos ) throws IOException; - SegmentPublishResult announceHistoricalSegments( + SegmentPublishResult commitReplaceSegments( Set segments, Set segmentsToDrop, - @Nullable DataSourceMetadata startMetadata, - @Nullable DataSourceMetadata endMetadata + @Nullable Set taskLockInfos ) throws IOException; /** @@ -324,11 +324,11 @@ SegmentPublishResult announceHistoricalSegments( int removeDataSourceMetadataOlderThan(long timestamp, @NotNull Set excludeDatasources); /** - * Similar to {@link #announceHistoricalSegments(Set)}, but meant for streaming ingestion tasks for handling + * Similar to {@link #commitSegments(Set)}, but meant for streaming ingestion tasks for handling * the case where the task ingested no records and created no segments, but still needs to update the metadata * with the progress that the task made. * - * The metadata should undergo the same validation checks as performed by {@link #announceHistoricalSegments}. + * The metadata should undergo the same validation checks as performed by {@link #commitSegments}. * * * @param dataSource the datasource diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index fb29fb3e8def..622ce842154d 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -297,9 +297,9 @@ private Collection retrieveAllUsedSegmentsForIntervalsWithHandle( } @Override - public Set announceHistoricalSegments(final Set segments) throws IOException + public Set commitSegments(final Set segments) throws IOException { - final SegmentPublishResult result = announceHistoricalSegments(segments, null, null, null, null, null, false); + final SegmentPublishResult result = commitSegments(segments, null, null, null); // Metadata transaction cannot fail because we are not trying to do one. if (!result.isSuccess()) { @@ -310,14 +310,11 @@ public Set announceHistoricalSegments(final Set segmen } @Override - public SegmentPublishResult announceHistoricalSegments( + public SegmentPublishResult commitSegments( final Set segments, final Set segmentsToDrop, @Nullable final DataSourceMetadata startMetadata, - @Nullable DataSourceMetadata endMetadata, - @Nullable Map segmentLockMap, - @Nullable Set taskLockInfos, - boolean append + @Nullable final DataSourceMetadata endMetadata ) throws IOException { if (segments.isEmpty()) { @@ -336,57 +333,15 @@ public SegmentPublishResult announceHistoricalSegments( } // Find which segments are used (i.e. not overshadowed). - Set usedSegments = new HashSet<>(); - Set newSegments = new HashSet<>(segments); - if (!append) { - List> segmentHolders = - SegmentTimeline.forSegments(segments).lookupWithIncompletePartitions(Intervals.ETERNITY); - for (TimelineObjectHolder holder : segmentHolders) { - for (PartitionChunk chunk : holder.getObject()) { - usedSegments.add(chunk.getObject()); - } - } - } else { - final Map> segmentToNewMetadataMap = connector.retryTransaction( - new TransactionCallback>>() - { - @Override - public Map> inTransaction( - final Handle handle, - final TransactionStatus transactionStatus - ) throws Exception - { - return allocateNewSegmentIds( - handle, - dataSource, - segments - ); - } - }, - 0, - SQLMetadataConnector.DEFAULT_MAX_TRIES - ); - for (DataSegment segment : segmentToNewMetadataMap.keySet()) { - for (SegmentIdWithShardSpec newId : segmentToNewMetadataMap.get(segment)) { - DataSegment newSegment = new DataSegment( - newId.getDataSource(), - newId.getInterval(), - newId.getVersion(), - segment.getLoadSpec(), - segment.getDimensions(), - segment.getMetrics(), - newId.getShardSpec(), - segment.getBinaryVersion(), - segment.getSize() - ); - newSegments.add(newSegment); - } + final Set usedSegments = new HashSet<>(); + List> segmentHolders = + SegmentTimeline.forSegments(segments).lookupWithIncompletePartitions(Intervals.ETERNITY); + for (TimelineObjectHolder holder : segmentHolders) { + for (PartitionChunk chunk : holder.getObject()) { + usedSegments.add(chunk.getObject()); } - usedSegments.addAll(newSegments); } - - final AtomicBoolean definitelyNotUpdated = new AtomicBoolean(false); try { @@ -442,7 +397,7 @@ public SegmentPublishResult inTransaction( } } - final Set inserted = announceHistoricalSegmentBatch(handle, newSegments, usedSegments, segmentLockMap, taskLockInfos, append); + final Set inserted = announceHistoricalSegmentBatch(handle, segments, usedSegments); return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); } @@ -462,14 +417,207 @@ public SegmentPublishResult inTransaction( } @Override - public SegmentPublishResult announceHistoricalSegments( - Set segments, - Set segmentsToDrop, - @Nullable DataSourceMetadata startMetadata, - @Nullable DataSourceMetadata endMetadata - ) throws IOException + public SegmentPublishResult commitReplaceSegments( + final Set segments, + final Set segmentsToDrop, + @Nullable Set taskLockInfos + ) { - return announceHistoricalSegments(segments, segmentsToDrop, startMetadata, endMetadata, null, null, false); + if (segments.isEmpty()) { + throw new IllegalArgumentException("segment set must not be empty"); + } + + final String dataSource = segments.iterator().next().getDataSource(); + for (DataSegment segment : segments) { + if (!dataSource.equals(segment.getDataSource())) { + throw new IllegalArgumentException("segments must all be from the same dataSource"); + } + } + + // Find which segments are used (i.e. not overshadowed). + Set usedSegments = new HashSet<>(); + Set newSegments = new HashSet<>(segments); + List> segmentHolders = + SegmentTimeline.forSegments(segments).lookupWithIncompletePartitions(Intervals.ETERNITY); + for (TimelineObjectHolder holder : segmentHolders) { + for (PartitionChunk chunk : holder.getObject()) { + usedSegments.add(chunk.getObject()); + } + } + + final AtomicBoolean definitelyNotUpdated = new AtomicBoolean(false); + + try { + return connector.retryTransaction( + new TransactionCallback() + { + @Override + public SegmentPublishResult inTransaction( + final Handle handle, + final TransactionStatus transactionStatus + ) throws Exception + { + // Set definitelyNotUpdated back to false upon retrying. + definitelyNotUpdated.set(false); + + + if (segmentsToDrop != null && !segmentsToDrop.isEmpty()) { + final DataStoreMetadataUpdateResult result = dropSegmentsWithHandle( + handle, + segmentsToDrop, + dataSource + ); + if (result.isFailed()) { + // Metadata store was definitely not updated. + transactionStatus.setRollbackOnly(); + definitelyNotUpdated.set(true); + + if (result.canRetry()) { + throw new RetryTransactionException(result.getErrorMsg()); + } else { + throw new RuntimeException(result.getErrorMsg()); + } + } + } + + final Set inserted = commitReplaceSegmentBatch(handle, newSegments, usedSegments, taskLockInfos); + + return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); + } + }, + 3, + getSqlMetadataMaxRetry() + ); + } + catch (CallbackFailedException e) { + if (definitelyNotUpdated.get()) { + return SegmentPublishResult.fail(e.getMessage()); + } else { + // Must throw exception if we are not sure if we updated or not. + throw e; + } + } + } + + @Override + public SegmentPublishResult commitAppendSegments( + final Set segments, + @Nullable final DataSourceMetadata startMetadata, + @Nullable DataSourceMetadata endMetadata, + @Nullable Map segmentLockMap, + @Nullable Set taskLockInfos + ) + { + if (segments.isEmpty()) { + throw new IllegalArgumentException("segment set must not be empty"); + } + + final String dataSource = segments.iterator().next().getDataSource(); + for (DataSegment segment : segments) { + if (!dataSource.equals(segment.getDataSource())) { + throw new IllegalArgumentException("segments must all be from the same dataSource"); + } + } + + if ((startMetadata == null && endMetadata != null) || (startMetadata != null && endMetadata == null)) { + throw new IllegalArgumentException("start/end metadata pair must be either null or non-null"); + } + + // Find which segments are used (i.e. not overshadowed). + Set newSegments = new HashSet<>(segments); + final Map> segmentToNewMetadataMap = connector.retryTransaction( + new TransactionCallback>>() + { + @Override + public Map> inTransaction( + final Handle handle, + final TransactionStatus transactionStatus + ) throws Exception + { + return allocateNewSegmentIds( + handle, + dataSource, + segments + ); + } + }, + 0, + SQLMetadataConnector.DEFAULT_MAX_TRIES + ); + for (DataSegment segment : segmentToNewMetadataMap.keySet()) { + for (SegmentIdWithShardSpec newId : segmentToNewMetadataMap.get(segment)) { + DataSegment newSegment = new DataSegment( + newId.getDataSource(), + newId.getInterval(), + newId.getVersion(), + segment.getLoadSpec(), + segment.getDimensions(), + segment.getMetrics(), + newId.getShardSpec(), + segment.getBinaryVersion(), + segment.getSize() + ); + newSegments.add(newSegment); + } + } + Set usedSegments = new HashSet<>(newSegments); + + + + final AtomicBoolean definitelyNotUpdated = new AtomicBoolean(false); + + try { + return connector.retryTransaction( + new TransactionCallback() + { + @Override + public SegmentPublishResult inTransaction( + final Handle handle, + final TransactionStatus transactionStatus + ) throws Exception + { + // Set definitelyNotUpdated back to false upon retrying. + definitelyNotUpdated.set(false); + + if (startMetadata != null) { + final DataStoreMetadataUpdateResult result = updateDataSourceMetadataWithHandle( + handle, + dataSource, + startMetadata, + endMetadata + ); + + if (result.isFailed()) { + // Metadata was definitely not updated. + transactionStatus.setRollbackOnly(); + definitelyNotUpdated.set(true); + + if (result.canRetry()) { + throw new RetryTransactionException(result.getErrorMsg()); + } else { + throw new RuntimeException(result.getErrorMsg()); + } + } + } + + + final Set inserted = commitAppendSegmentBatch(handle, newSegments, usedSegments, segmentLockMap); + + return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); + } + }, + 3, + getSqlMetadataMaxRetry() + ); + } + catch (CallbackFailedException e) { + if (definitelyNotUpdated.get()) { + return SegmentPublishResult.fail(e.getMessage()); + } else { + // Must throw exception if we are not sure if we updated or not. + throw e; + } + } } @Override @@ -1550,13 +1698,334 @@ public int deletePendingSegments(String dataSource) ); } - /** - * Attempts to insert a single segment to the database. If the segment already exists, will do nothing; although, - * this checking is imperfect and callers must be prepared to retry their entire transaction on exceptions. - * - * @return DataSegment set inserted - */ private Set announceHistoricalSegmentBatch( + final Handle handle, + final Set segments, + final Set usedSegments + ) throws IOException + { + final Set toInsertSegments = new HashSet<>(); + try { + Set existedSegments = segmentExistsBatch(handle, segments); + log.info("Found these segments already exist in DB: %s", existedSegments); + for (DataSegment segment : segments) { + if (!existedSegments.contains(segment.getId().toString())) { + toInsertSegments.add(segment); + } + } + + // SELECT -> INSERT can fail due to races; callers must be prepared to retry. + // Avoiding ON DUPLICATE KEY since it's not portable. + // Avoiding try/catch since it may cause inadvertent transaction-splitting. + final List> partitionedSegments = Lists.partition( + new ArrayList<>(toInsertSegments), + MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE + ); + + PreparedBatch preparedBatch = handle.prepareBatch( + StringUtils.format( + "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", + dbTables.getSegmentsTable(), + connector.getQuoteString() + ) + ); + + for (List partition : partitionedSegments) { + for (DataSegment segment : partition) { + preparedBatch.add() + .bind("id", segment.getId().toString()) + .bind("dataSource", segment.getDataSource()) + .bind("created_date", DateTimes.nowUtc().toString()) + .bind("start", segment.getInterval().getStart().toString()) + .bind("end", segment.getInterval().getEnd().toString()) + .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) + .bind("version", segment.getVersion()) + .bind("used", usedSegments.contains(segment)) + .bind("payload", jsonMapper.writeValueAsBytes(segment)); + } + final int[] affectedRows = preparedBatch.execute(); + final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); + if (succeeded) { + log.infoSegments(partition, "Published segments to DB"); + } else { + final List failedToPublish = IntStream.range(0, partition.size()) + .filter(i -> affectedRows[i] != 1) + .mapToObj(partition::get) + .collect(Collectors.toList()); + throw new ISE( + "Failed to publish segments to DB: %s", + SegmentUtils.commaSeparatedIdentifiers(failedToPublish) + ); + } + } + } + catch (Exception e) { + log.errorSegments(segments, "Exception inserting segments"); + throw e; + } + + return toInsertSegments; + } + + private Set commitReplaceSegmentBatch( + final Handle handle, + final Set segments, + final Set usedSegments, + @Nullable Set replaceLocks + ) throws IOException + { + final Set toInsertSegments = new HashSet<>(); + try { + Set existedSegments = segmentExistsBatch(handle, segments); + log.info("Found these segments already exist in DB: %s", existedSegments); + for (DataSegment segment : segments) { + if (!existedSegments.contains(segment.getId().toString())) { + toInsertSegments.add(segment); + } + } + + // SELECT -> INSERT can fail due to races; callers must be prepared to retry. + // Avoiding ON DUPLICATE KEY since it's not portable. + // Avoiding try/catch since it may cause inadvertent transaction-splitting. + final List> partitionedSegments = Lists.partition( + new ArrayList<>(toInsertSegments), + MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE + ); + + PreparedBatch preparedBatch = handle.prepareBatch( + StringUtils.format( + "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", + dbTables.getSegmentsTable(), + connector.getQuoteString() + ) + ); + for (List partition : partitionedSegments) { + for (DataSegment segment : partition) { + preparedBatch.add() + .bind("id", segment.getId().toString()) + .bind("dataSource", segment.getDataSource()) + .bind("created_date", DateTimes.nowUtc().toString()) + .bind("start", segment.getInterval().getStart().toString()) + .bind("end", segment.getInterval().getEnd().toString()) + .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) + .bind("version", segment.getVersion()) + .bind("used", usedSegments.contains(segment)) + .bind("payload", jsonMapper.writeValueAsBytes(segment)); + } + final int[] affectedInsertRows = preparedBatch.execute(); + + final boolean succeeded = Arrays.stream(affectedInsertRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); + if (succeeded) { + log.infoSegments(partition, "Published segments to DB"); + } else { + final List failedToPublish = IntStream.range(0, partition.size()) + .filter(i -> affectedInsertRows[i] != 1) + .mapToObj(partition::get) + .collect(Collectors.toList()); + throw new ISE( + "Failed to publish segments to DB: %s", + SegmentUtils.commaSeparatedIdentifiers(failedToPublish) + ); + } + } + + PreparedBatch appendBatch = handle.prepareBatch( + StringUtils.format( + "INSERT INTO %1$s (id, dataSource, start, %2$send%2$s, segment_id, lock_version) " + + "VALUES (:id, :dataSource, :start, :end, :segment_id, :lock_version)", + dbTables.getSegmentVersionsTable(), + connector.getQuoteString() + ) + ); + Map segmentsToBeForwarded = getAppendedSegmentIds( + handle, + segments.iterator().next().getDataSource(), + replaceLocks + ); + final int numCorePartitions = segments.size(); + int partitionNum = segments.size(); + final List>> forwardSegmentsBatch = Lists.partition( + new ArrayList<>(segmentsToBeForwarded.entrySet()), + MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE + ); + for (List> batch : forwardSegmentsBatch) { + Map batchMap = new HashMap<>(); + for (Map.Entry entry : batch) { + batchMap.put(entry.getKey(), entry.getValue()); + } + List oldSegments = retrieveSegments(handle, batchMap.keySet()); + for (DataSegment oldSegment : oldSegments) { + Interval newInterval = oldSegment.getInterval(); + for (DataSegment segment : segments) { + if (segment.getInterval().overlaps(newInterval)) { + if (segment.getInterval().contains(newInterval)) { + newInterval = segment.getInterval(); + } else { + throw new ISE("Incompatible segment intervals for commit: [%s] and [%s].", + newInterval, + segment.getInterval() + ); + } + } + } + TaskLockInfo lock = batchMap.get(oldSegment.getId().toString()); + ShardSpec shardSpec = new NumberedShardSpec(partitionNum++, numCorePartitions); + DataSegment newSegment = new DataSegment( + oldSegment.getDataSource(), + newInterval, + lock.getVersion(), + oldSegment.getLoadSpec(), + oldSegment.getDimensions(), + oldSegment.getMetrics(), + shardSpec, + oldSegment.getBinaryVersion(), + oldSegment.getSize() + ); + preparedBatch.add() + .bind("id", newSegment.getId().toString()) + .bind("dataSource", newSegment.getDataSource()) + .bind("created_date", DateTimes.nowUtc().toString()) + .bind("start", newSegment.getInterval().getStart().toString()) + .bind("end", newSegment.getInterval().getEnd().toString()) + .bind("partitioned", (newSegment.getShardSpec() instanceof NoneShardSpec) ? false : true) + .bind("version", newSegment.getVersion()) + .bind("used", true) + .bind("payload", jsonMapper.writeValueAsBytes(newSegment)); + } + final int[] affectedInsertRows = preparedBatch.execute(); + + final boolean succeeded = Arrays.stream(affectedInsertRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); + if (succeeded) { + log.info("Published segments with updated metadata to DB"); + } else { + throw new ISE("Failed to update segment metadatas in DB"); + } + } + } + catch (Exception e) { + log.errorSegments(segments, "Exception inserting segment metadata"); + throw e; + } + + return toInsertSegments; + } + + private Set commitAppendSegmentBatch( + final Handle handle, + final Set segments, + final Set usedSegments, + @Nullable Map appendSegmentLockMap + ) throws IOException + { + final Set toInsertSegments = new HashSet<>(); + try { + Set existedSegments = segmentExistsBatch(handle, segments); + log.info("Found these segments already exist in DB: %s", existedSegments); + for (DataSegment segment : segments) { + if (!existedSegments.contains(segment.getId().toString())) { + toInsertSegments.add(segment); + } + } + + // SELECT -> INSERT can fail due to races; callers must be prepared to retry. + // Avoiding ON DUPLICATE KEY since it's not portable. + // Avoiding try/catch since it may cause inadvertent transaction-splitting. + final List> partitionedSegments = Lists.partition( + new ArrayList<>(toInsertSegments), + MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE + ); + + PreparedBatch preparedBatch = handle.prepareBatch( + StringUtils.format( + "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", + dbTables.getSegmentsTable(), + connector.getQuoteString() + ) + ); + for (List partition : partitionedSegments) { + for (DataSegment segment : partition) { + preparedBatch.add() + .bind("id", segment.getId().toString()) + .bind("dataSource", segment.getDataSource()) + .bind("created_date", DateTimes.nowUtc().toString()) + .bind("start", segment.getInterval().getStart().toString()) + .bind("end", segment.getInterval().getEnd().toString()) + .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) + .bind("version", segment.getVersion()) + .bind("used", usedSegments.contains(segment)) + .bind("payload", jsonMapper.writeValueAsBytes(segment)); + } + final int[] affectedInsertRows = preparedBatch.execute(); + + final boolean succeeded = Arrays.stream(affectedInsertRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); + if (succeeded) { + log.infoSegments(partition, "Published segments to DB"); + } else { + final List failedToPublish = IntStream.range(0, partition.size()) + .filter(i -> affectedInsertRows[i] != 1) + .mapToObj(partition::get) + .collect(Collectors.toList()); + throw new ISE( + "Failed to publish segments to DB: %s", + SegmentUtils.commaSeparatedIdentifiers(failedToPublish) + ); + } + } + + PreparedBatch appendBatch = handle.prepareBatch( + StringUtils.format( + "INSERT INTO %1$s (id, dataSource, start, %2$send%2$s, segment_id, lock_version) " + + "VALUES (:id, :dataSource, :start, :end, :segment_id, :lock_version)", + dbTables.getSegmentVersionsTable(), + connector.getQuoteString() + ) + ); + if (appendSegmentLockMap == null) { + appendSegmentLockMap = new HashMap<>(); + } + final List>> appendSegmentPartitions = Lists.partition( + new ArrayList<>(appendSegmentLockMap.entrySet()), + MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE + ); + for (List> partition : appendSegmentPartitions) { + for (Map.Entry entry : partition) { + DataSegment segment = entry.getKey(); + TaskLockInfo lock = entry.getValue(); + appendBatch.add() + .bind("id", segment.getId().toString() + ":" + lock.hashCode()) + .bind("dataSource", segment.getDataSource()) + .bind("start", lock.getInterval().getStartMillis()) + .bind("end", lock.getInterval().getEndMillis()) + .bind("segment_id", segment.getId().toString()) + .bind("lock_version", lock.getVersion()); + } + final int[] affectedAppendRows = appendBatch.execute(); + final boolean succeeded = Arrays.stream(affectedAppendRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); + if (!succeeded) { + final List failedToForward = IntStream.range(0, partition.size()) + .filter(i -> affectedAppendRows[i] != 1) + .mapToObj(partition::get) + .map(x -> x.getKey()) + .collect(Collectors.toList()); + throw new ISE( + "Failed to forward appended segments to DB: %s", + SegmentUtils.commaSeparatedIdentifiers(failedToForward) + ); + } + } + } + catch (Exception e) { + log.errorSegments(segments, "Exception inserting segment metadata"); + throw e; + } + + return toInsertSegments; + } + + private Set commitSegmentBatch( final Handle handle, final Set segments, final Set usedSegments, @@ -1902,7 +2371,7 @@ private Set segmentExistsBatch(final Handle handle, final Set dataSegments) @Test public void testSimpleAnnounce() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); for (DataSegment segment : SEGMENTS) { Assert.assertArrayEquals( mapper.writeValueAsString(segment).getBytes(StandardCharsets.UTF_8), @@ -507,7 +507,7 @@ public void testAnnounceHistoricalSegments() throws IOException ); } - coordinator.announceHistoricalSegments(segments); + coordinator.commitSegments(segments); for (DataSegment segment : segments) { Assert.assertArrayEquals( mapper.writeValueAsString(segment).getBytes(StandardCharsets.UTF_8), @@ -536,7 +536,7 @@ public void testOvershadowingAnnounce() throws IOException { final ImmutableSet segments = ImmutableSet.of(defaultSegment, defaultSegment2, defaultSegment4); - coordinator.announceHistoricalSegments(segments); + coordinator.commitSegments(segments); for (DataSegment segment : segments) { Assert.assertArrayEquals( @@ -557,7 +557,7 @@ public void testOvershadowingAnnounce() throws IOException public void testTransactionalAnnounceSuccess() throws IOException { // Insert first segment. - final SegmentPublishResult result1 = coordinator.announceHistoricalSegments( + final SegmentPublishResult result1 = coordinator.commitSegments( ImmutableSet.of(defaultSegment), ImmutableSet.of(), new ObjectMetadata(null), @@ -576,7 +576,7 @@ public void testTransactionalAnnounceSuccess() throws IOException ); // Insert second segment. - final SegmentPublishResult result2 = coordinator.announceHistoricalSegments( + final SegmentPublishResult result2 = coordinator.commitSegments( ImmutableSet.of(defaultSegment2), ImmutableSet.of(), new ObjectMetadata(ImmutableMap.of("foo", "bar")), @@ -633,7 +633,7 @@ protected DataStoreMetadataUpdateResult updateDataSourceMetadataWithHandle( }; // Insert first segment. - final SegmentPublishResult result1 = failOnceCoordinator.announceHistoricalSegments( + final SegmentPublishResult result1 = failOnceCoordinator.commitSegments( ImmutableSet.of(defaultSegment), ImmutableSet.of(), new ObjectMetadata(null), @@ -655,7 +655,7 @@ protected DataStoreMetadataUpdateResult updateDataSourceMetadataWithHandle( attemptCounter.set(0); // Insert second segment. - final SegmentPublishResult result2 = failOnceCoordinator.announceHistoricalSegments( + final SegmentPublishResult result2 = failOnceCoordinator.commitSegments( ImmutableSet.of(defaultSegment2), ImmutableSet.of(), new ObjectMetadata(ImmutableMap.of("foo", "bar")), @@ -686,7 +686,7 @@ protected DataStoreMetadataUpdateResult updateDataSourceMetadataWithHandle( @Test public void testTransactionalAnnounceFailDbNullWantNotNull() throws IOException { - final SegmentPublishResult result1 = coordinator.announceHistoricalSegments( + final SegmentPublishResult result1 = coordinator.commitSegments( ImmutableSet.of(defaultSegment), ImmutableSet.of(), new ObjectMetadata(ImmutableMap.of("foo", "bar")), @@ -720,7 +720,7 @@ public void testTransactionalAnnounceFailSegmentDropFailWithoutRetry() throws IO .build(); Set dropSegments = ImmutableSet.of(existingSegment1, existingSegment2, dataSegmentBar); - final SegmentPublishResult result1 = coordinator.announceHistoricalSegments( + final SegmentPublishResult result1 = coordinator.commitSegments( SEGMENTS, dropSegments, null, @@ -749,7 +749,7 @@ public void testTransactionalAnnounceSucceedWithSegmentDrop() throws IOException retrieveUsedSegmentIds() ); - final SegmentPublishResult result1 = coordinator.announceHistoricalSegments( + final SegmentPublishResult result1 = coordinator.commitSegments( SEGMENTS, ImmutableSet.of(existingSegment1, existingSegment2), null, @@ -787,7 +787,7 @@ public void testTransactionalAnnounceFailSegmentDropFailWithRetry() throws IOExc ); Set dropSegments = ImmutableSet.of(existingSegment1, defaultSegment4); - final SegmentPublishResult result1 = coordinator.announceHistoricalSegments( + final SegmentPublishResult result1 = coordinator.commitSegments( SEGMENTS, dropSegments, null, @@ -808,7 +808,7 @@ public void testTransactionalAnnounceFailSegmentDropFailWithRetry() throws IOExc @Test public void testTransactionalAnnounceFailDbNotNullWantNull() throws IOException { - final SegmentPublishResult result1 = coordinator.announceHistoricalSegments( + final SegmentPublishResult result1 = coordinator.commitSegments( ImmutableSet.of(defaultSegment), ImmutableSet.of(), new ObjectMetadata(null), @@ -816,7 +816,7 @@ public void testTransactionalAnnounceFailDbNotNullWantNull() throws IOException ); Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(defaultSegment)), result1); - final SegmentPublishResult result2 = coordinator.announceHistoricalSegments( + final SegmentPublishResult result2 = coordinator.commitSegments( ImmutableSet.of(defaultSegment2), ImmutableSet.of(), new ObjectMetadata(null), @@ -834,7 +834,7 @@ public void testTransactionalAnnounceFailDbNotNullWantNull() throws IOException @Test public void testTransactionalAnnounceFailDbNotNullWantDifferent() throws IOException { - final SegmentPublishResult result1 = coordinator.announceHistoricalSegments( + final SegmentPublishResult result1 = coordinator.commitSegments( ImmutableSet.of(defaultSegment), ImmutableSet.of(), new ObjectMetadata(null), @@ -842,7 +842,7 @@ public void testTransactionalAnnounceFailDbNotNullWantDifferent() throws IOExcep ); Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(defaultSegment)), result1); - final SegmentPublishResult result2 = coordinator.announceHistoricalSegments( + final SegmentPublishResult result2 = coordinator.commitSegments( ImmutableSet.of(defaultSegment2), ImmutableSet.of(), new ObjectMetadata(ImmutableMap.of("foo", "qux")), @@ -860,7 +860,7 @@ public void testTransactionalAnnounceFailDbNotNullWantDifferent() throws IOExcep @Test public void testSimpleUsedList() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); Assert.assertEquals( SEGMENTS, ImmutableSet.copyOf( @@ -876,8 +876,8 @@ public void testSimpleUsedList() throws IOException @Test public void testMultiIntervalUsedList() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); - coordinator.announceHistoricalSegments(ImmutableSet.of(defaultSegment3)); + coordinator.commitSegments(SEGMENTS); + coordinator.commitSegments(ImmutableSet.of(defaultSegment3)); Assertions.assertThat( coordinator.retrieveUsedSegmentsForIntervals( @@ -919,7 +919,7 @@ public void testMultiIntervalUsedList() throws IOException @Test public void testSimpleUnusedList() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); markAllSegmentsUnused(); Assert.assertEquals( SEGMENTS, @@ -936,7 +936,7 @@ public void testSimpleUnusedList() throws IOException @Test public void testUsedOverlapLow() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); Set actualSegments = ImmutableSet.copyOf( coordinator.retrieveUsedSegmentsForInterval( defaultSegment.getDataSource(), @@ -954,7 +954,7 @@ public void testUsedOverlapLow() throws IOException @Test public void testUsedOverlapHigh() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); Assert.assertEquals( SEGMENTS, ImmutableSet.copyOf( @@ -970,7 +970,7 @@ public void testUsedOverlapHigh() throws IOException @Test public void testUsedOutOfBoundsLow() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); Assert.assertTrue( coordinator.retrieveUsedSegmentsForInterval( defaultSegment.getDataSource(), @@ -984,7 +984,7 @@ public void testUsedOutOfBoundsLow() throws IOException @Test public void testUsedOutOfBoundsHigh() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); Assert.assertTrue( coordinator.retrieveUsedSegmentsForInterval( defaultSegment.getDataSource(), @@ -997,7 +997,7 @@ public void testUsedOutOfBoundsHigh() throws IOException @Test public void testUsedWithinBoundsEnd() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); Assert.assertEquals( SEGMENTS, ImmutableSet.copyOf( @@ -1013,7 +1013,7 @@ public void testUsedWithinBoundsEnd() throws IOException @Test public void testUsedOverlapEnd() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); Assert.assertEquals( SEGMENTS, ImmutableSet.copyOf( @@ -1030,7 +1030,7 @@ public void testUsedOverlapEnd() throws IOException @Test public void testUnusedOverlapLow() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); markAllSegmentsUnused(); Assert.assertTrue( coordinator.retrieveUnusedSegmentsForInterval( @@ -1046,7 +1046,7 @@ public void testUnusedOverlapLow() throws IOException @Test public void testUnusedUnderlapLow() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); markAllSegmentsUnused(); Assert.assertTrue( coordinator.retrieveUnusedSegmentsForInterval( @@ -1060,7 +1060,7 @@ public void testUnusedUnderlapLow() throws IOException @Test public void testUnusedUnderlapHigh() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); markAllSegmentsUnused(); Assert.assertTrue( coordinator.retrieveUnusedSegmentsForInterval( @@ -1073,7 +1073,7 @@ public void testUnusedUnderlapHigh() throws IOException @Test public void testUnusedOverlapHigh() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); markAllSegmentsUnused(); Assert.assertTrue( coordinator.retrieveUnusedSegmentsForInterval( @@ -1086,7 +1086,7 @@ public void testUnusedOverlapHigh() throws IOException @Test public void testUnusedBigOverlap() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); markAllSegmentsUnused(); Assert.assertEquals( SEGMENTS, @@ -1102,7 +1102,7 @@ public void testUnusedBigOverlap() throws IOException @Test public void testUnusedLowRange() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); markAllSegmentsUnused(); Assert.assertEquals( SEGMENTS, @@ -1127,7 +1127,7 @@ public void testUnusedLowRange() throws IOException @Test public void testUnusedHighRange() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); markAllSegmentsUnused(); Assert.assertEquals( SEGMENTS, @@ -1152,7 +1152,7 @@ public void testUnusedHighRange() throws IOException @Test public void testUsedHugeTimeRangeEternityFilter() throws IOException { - coordinator.announceHistoricalSegments( + coordinator.commitSegments( ImmutableSet.of( hugeTimeRangeSegment1, hugeTimeRangeSegment2, @@ -1175,7 +1175,7 @@ public void testUsedHugeTimeRangeEternityFilter() throws IOException @Test public void testUsedHugeTimeRangeTrickyFilter1() throws IOException { - coordinator.announceHistoricalSegments( + coordinator.commitSegments( ImmutableSet.of( hugeTimeRangeSegment1, hugeTimeRangeSegment2, @@ -1198,7 +1198,7 @@ public void testUsedHugeTimeRangeTrickyFilter1() throws IOException @Test public void testUsedHugeTimeRangeTrickyFilter2() throws IOException { - coordinator.announceHistoricalSegments( + coordinator.commitSegments( ImmutableSet.of( hugeTimeRangeSegment1, hugeTimeRangeSegment2, @@ -1222,7 +1222,7 @@ public void testUsedHugeTimeRangeTrickyFilter2() throws IOException @Test public void testEternitySegmentWithStringComparison() throws IOException { - coordinator.announceHistoricalSegments( + coordinator.commitSegments( ImmutableSet.of( eternitySegment ) @@ -1243,7 +1243,7 @@ public void testEternitySegmentWithStringComparison() throws IOException @Test public void testEternityMultipleSegmentWithStringComparison() throws IOException { - coordinator.announceHistoricalSegments( + coordinator.commitSegments( ImmutableSet.of( numberedSegment0of0, eternitySegment @@ -1265,7 +1265,7 @@ public void testEternityMultipleSegmentWithStringComparison() throws IOException @Test public void testFirstHalfEternitySegmentWithStringComparison() throws IOException { - coordinator.announceHistoricalSegments( + coordinator.commitSegments( ImmutableSet.of( firstHalfEternityRangeSegment ) @@ -1286,7 +1286,7 @@ public void testFirstHalfEternitySegmentWithStringComparison() throws IOExceptio @Test public void testFirstHalfEternityMultipleSegmentWithStringComparison() throws IOException { - coordinator.announceHistoricalSegments( + coordinator.commitSegments( ImmutableSet.of( numberedSegment0of0, firstHalfEternityRangeSegment @@ -1308,7 +1308,7 @@ public void testFirstHalfEternityMultipleSegmentWithStringComparison() throws IO @Test public void testSecondHalfEternitySegmentWithStringComparison() throws IOException { - coordinator.announceHistoricalSegments( + coordinator.commitSegments( ImmutableSet.of( secondHalfEternityRangeSegment ) @@ -1331,7 +1331,7 @@ public void testSecondHalfEternitySegmentWithStringComparison() throws IOExcepti @Test public void testLargeIntervalWithStringComparison() throws IOException { - coordinator.announceHistoricalSegments( + coordinator.commitSegments( ImmutableSet.of( hugeTimeRangeSegment4 ) @@ -1352,7 +1352,7 @@ public void testLargeIntervalWithStringComparison() throws IOException @Test public void testSecondHalfEternityMultipleSegmentWithStringComparison() throws IOException { - coordinator.announceHistoricalSegments( + coordinator.commitSegments( ImmutableSet.of( numberedSegment0of0, secondHalfEternityRangeSegment @@ -1374,7 +1374,7 @@ public void testSecondHalfEternityMultipleSegmentWithStringComparison() throws I @Test public void testDeleteDataSourceMetadata() throws IOException { - coordinator.announceHistoricalSegments( + coordinator.commitSegments( ImmutableSet.of(defaultSegment), ImmutableSet.of(), new ObjectMetadata(null), @@ -1396,7 +1396,7 @@ public void testDeleteDataSourceMetadata() throws IOException public void testDeleteSegmentsInMetaDataStorage() throws IOException { // Published segments to MetaDataStorage - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); // check segments Published Assert.assertEquals( @@ -1429,7 +1429,7 @@ public void testDeleteSegmentsInMetaDataStorage() throws IOException public void testUpdateSegmentsInMetaDataStorage() throws IOException { // Published segments to MetaDataStorage - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); // check segments Published Assert.assertEquals( @@ -1491,7 +1491,7 @@ public void testMultipleAdditionalNumberedShardsWithOneCorePartition() throws IO private void additionalNumberedShardTest(Set segments) throws IOException { - coordinator.announceHistoricalSegments(segments); + coordinator.commitSegments(segments); for (DataSegment segment : segments) { Assert.assertArrayEquals( @@ -2083,7 +2083,7 @@ public void testAllocatePendingSegmentsWithOvershadowingSegments() throws IOExce 10L ) ); - final Set announced = coordinator.announceHistoricalSegments(toBeAnnounced); + final Set announced = coordinator.commitSegments(toBeAnnounced); Assert.assertEquals(toBeAnnounced, announced); } @@ -2136,7 +2136,7 @@ public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IO Assert.assertEquals(0, shardSpec.getNumCorePartitions()); Assert.assertEquals(5, shardSpec.getNumBuckets()); - coordinator.announceHistoricalSegments( + coordinator.commitSegments( Collections.singleton( new DataSegment( id.getDataSource(), @@ -2167,7 +2167,7 @@ public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IO Assert.assertEquals(0, shardSpec.getNumCorePartitions()); Assert.assertEquals(5, shardSpec.getNumBuckets()); - coordinator.announceHistoricalSegments( + coordinator.commitSegments( Collections.singleton( new DataSegment( id.getDataSource(), @@ -2229,7 +2229,7 @@ public void testAddNumberedShardSpecAfterMultiDimensionsShardSpecWithUnknownCore ) ); } - coordinator.announceHistoricalSegments(originalSegments); + coordinator.commitSegments(originalSegments); final SegmentIdWithShardSpec id = coordinator.allocatePendingSegment( datasource, "seq", @@ -2274,7 +2274,7 @@ public void testAddNumberedShardSpecAfterSingleDimensionsShardSpecWithUnknownCor ) ); } - coordinator.announceHistoricalSegments(originalSegments); + coordinator.commitSegments(originalSegments); final SegmentIdWithShardSpec id = coordinator.allocatePendingSegment( datasource, "seq", @@ -2330,7 +2330,7 @@ public void testDropSegmentsWithHandleForSegmentThatDoesNotExist() @Test public void testRemoveDataSourceMetadataOlderThanDatasourceActiveShouldNotBeDeleted() throws Exception { - coordinator.announceHistoricalSegments( + coordinator.commitSegments( ImmutableSet.of(defaultSegment), ImmutableSet.of(), new ObjectMetadata(null), @@ -2359,7 +2359,7 @@ public void testRemoveDataSourceMetadataOlderThanDatasourceActiveShouldNotBeDele @Test public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveAndOlderThanTimeShouldBeDeleted() throws Exception { - coordinator.announceHistoricalSegments( + coordinator.commitSegments( ImmutableSet.of(defaultSegment), ImmutableSet.of(), new ObjectMetadata(null), @@ -2385,7 +2385,7 @@ public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveAndOlderThan public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveButNotOlderThanTimeShouldNotBeDeleted() throws Exception { - coordinator.announceHistoricalSegments( + coordinator.commitSegments( ImmutableSet.of(defaultSegment), ImmutableSet.of(), new ObjectMetadata(null), @@ -2415,7 +2415,7 @@ public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveButNotOlderT @Test public void testMarkSegmentsAsUnusedWithinIntervalOneYear() throws IOException { - coordinator.announceHistoricalSegments(ImmutableSet.of(existingSegment1, existingSegment2)); + coordinator.commitSegments(ImmutableSet.of(existingSegment1, existingSegment2)); // interval covers existingSegment1 and partially overlaps existingSegment2, // only existingSegment1 will be dropped @@ -2447,7 +2447,7 @@ public void testMarkSegmentsAsUnusedWithinIntervalOneYear() throws IOException @Test public void testMarkSegmentsAsUnusedWithinIntervalTwoYears() throws IOException { - coordinator.announceHistoricalSegments(ImmutableSet.of(existingSegment1, existingSegment2)); + coordinator.commitSegments(ImmutableSet.of(existingSegment1, existingSegment2)); // interval covers existingSegment1 and partially overlaps existingSegment2, // only existingSegment1 will be dropped From e3e9cf30dd1ebbbf03a7664aea0d3d543d33f683 Mon Sep 17 00:00:00 2001 From: Amatya Date: Mon, 7 Aug 2023 11:26:11 +0530 Subject: [PATCH 06/43] Fix version logic --- .../java/org/apache/druid/indexing/common/task/IndexTask.java | 1 + .../apache/druid/indexing/overlord/TimeChunkLockRequest.java | 3 +++ 2 files changed, 4 insertions(+) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index ebc728f9d87a..ab59c083a143 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -54,6 +54,7 @@ import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.CommitMetadataAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java index b99d14a8e4fe..e9871da543ba 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java @@ -24,6 +24,7 @@ import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TimeChunkLock; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.java.util.common.DateTimes; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -79,6 +80,8 @@ public TimeChunkLockRequest( this.interval = interval; if (lockType.equals(TaskLockType.APPEND) && preferredVersion == null) { this.preferredVersion = MIN_VERSION; + } else if (preferredVersion == null) { + this.preferredVersion = DateTimes.nowUtc().toString(); } else { this.preferredVersion = preferredVersion; } From 2ed61fdb24ac9b795104b46d6d094f6f5e63db8a Mon Sep 17 00:00:00 2001 From: Amatya Date: Tue, 8 Aug 2023 11:30:31 +0530 Subject: [PATCH 07/43] Fix checkstyle --- .../java/org/apache/druid/indexing/common/task/IndexTask.java | 1 - 1 file changed, 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index ab59c083a143..ebc728f9d87a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -54,7 +54,6 @@ import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.CommitMetadataAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; From 38a0071190549e8fbcd0cafdf47b6d02f25da150 Mon Sep 17 00:00:00 2001 From: Amatya Date: Wed, 16 Aug 2023 21:13:02 +0530 Subject: [PATCH 08/43] Fix test --- .../metadata/IndexerSQLMetadataStorageCoordinatorTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 33dbd87eb4e0..8b0e8202724e 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -935,7 +935,7 @@ public void testSimpleUnusedList() throws IOException @Test public void testSimpleUnusedListWithLimit() throws IOException { - coordinator.announceHistoricalSegments(SEGMENTS); + coordinator.commitSegments(SEGMENTS); markAllSegmentsUnused(); int limit = SEGMENTS.size() - 1; Set retreivedUnusedSegments = ImmutableSet.copyOf( From f5144a0974639219f573ddb4690805b99ea40106 Mon Sep 17 00:00:00 2001 From: Amatya Date: Thu, 17 Aug 2023 15:26:51 +0530 Subject: [PATCH 09/43] Handle lock type when not specified in context --- .../common/task/AppenderatorDriverRealtimeIndexTask.java | 4 +++- .../apache/druid/indexing/common/task/TaskLockHelper.java | 7 ++++--- .../task/batch/parallel/ParallelIndexSupervisorTask.java | 4 +++- .../batch/parallel/SinglePhaseParallelIndexingTest.java | 4 ++-- 4 files changed, 12 insertions(+), 7 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index 897981f34c16..7b09e9a7958e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -292,7 +292,9 @@ public TaskStatus runTask(final TaskToolbox toolbox) DiscoveryDruidNode discoveryDruidNode = createDiscoveryDruidNode(toolbox); appenderator = newAppenderator(dataSchema, tuningConfig, metrics, toolbox); - TaskLockType lockType = getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE); + final TaskLockType lockType = TaskLockType.valueOf( + getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name()) + ); StreamAppenderatorDriver driver = newDriver(dataSchema, appenderator, toolbox, metrics, lockType); try { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskLockHelper.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskLockHelper.java index 77e5b66f969d..84f6f3e1f45e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskLockHelper.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskLockHelper.java @@ -323,11 +323,12 @@ public static TaskLockType lockTypeFrom( } final TaskLockType lockTypeVal; - final String lockTypeName = context.get(Tasks.TASK_LOCK_TYPE).toString(); + final Object lockTypeName = context.get(Tasks.TASK_LOCK_TYPE); if (lockTypeName == null) { - lockTypeVal = (boolean) context.get(Tasks.USE_SHARED_LOCK) ? TaskLockType.SHARED : TaskLockType.EXCLUSIVE; + final Object useSharedLock = context.getOrDefault(Tasks.USE_SHARED_LOCK, false); + lockTypeVal = (boolean) useSharedLock ? TaskLockType.SHARED : TaskLockType.EXCLUSIVE; } else { - lockTypeVal = TaskLockType.valueOf(lockTypeName); + lockTypeVal = TaskLockType.valueOf(lockTypeName.toString()); } final Set appendModeCompatible = ImmutableSet.of(TaskLockType.SHARED, TaskLockType.APPEND); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 8d170cb62372..d3536c41c686 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -1172,7 +1172,9 @@ private void publishSegments( } final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish, commitMetadata) -> { - TaskLockType lockType = TaskLockType.valueOf(getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name())); + final TaskLockType lockType = TaskLockType.valueOf( + getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name()) + ); switch (lockType) { case REPLACE: return toolbox.getTaskActionClient().submit( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index c5d9482a3d33..950983d77b85 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -621,9 +621,9 @@ public void testMultipleAppends() final ParallelIndexSupervisorTask task = newTask(interval, Granularities.DAY, true, true); final ParallelIndexSupervisorTask task2 = newTask(interval, Granularities.DAY, true, true); task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); - task.addToContext(Tasks.TASK_LOCK_TYPE, TaskLockType.SHARED); + task.addToContext(Tasks.USE_SHARED_LOCK, true); task2.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); - task2.addToContext(Tasks.TASK_LOCK_TYPE, TaskLockType.SHARED); + task2.addToContext(Tasks.USE_SHARED_LOCK, true); getIndexingServiceClient().runTask(task.getId(), task); getIndexingServiceClient().runTask(task2.getId(), task2); From 1413a303ca9065343867cc581fba7786ff165877 Mon Sep 17 00:00:00 2001 From: Amatya Date: Fri, 18 Aug 2023 11:25:51 +0530 Subject: [PATCH 10/43] Use Intervals.utc --- .../druid/metadata/IndexerSQLMetadataStorageCoordinator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 7dc92974d3be..ab849cf1ab27 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -2301,7 +2301,7 @@ private Map getAppendedSegmentIds( final ResultIterator> resultIterator = query.map((index, r, ctx) -> { String segmentId = r.getString("segment_id"); - Interval interval = new Interval(r.getLong("start"), r.getLong("end")); + Interval interval = Intervals.utc(r.getLong("start"), r.getLong("end")); String version = r.getString("lock_version"); return Pair.of(segmentId, new TaskLockInfo(interval, version)); }).iterator(); From d495c3c7985d94290ef54068e80c8ac3d7561b77 Mon Sep 17 00:00:00 2001 From: Amatya Date: Mon, 21 Aug 2023 09:33:09 +0530 Subject: [PATCH 11/43] Modify segment commit sql statement --- .../SegmentTransactionalAppendAction.java | 2 +- .../SegmentTransactionalReplaceAction.java | 2 +- .../common/task/KillUnusedSegmentsTask.java | 20 ++++- .../IndexerSQLMetadataStorageCoordinator.java | 82 +++++++------------ 4 files changed, 52 insertions(+), 54 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java index e0d155408dc0..a0e9a304e826 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java @@ -200,7 +200,7 @@ public boolean isAudited() @Override public String toString() { - return "SegmentTransactionalInsertAction{" + + return "SegmentTransactionalAppendAction{" + "segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + ", startMetadata=" + startMetadata + ", endMetadata=" + endMetadata + diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index 4dd3ea7a389a..907cf8c7cd09 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -197,7 +197,7 @@ public boolean isAudited() @Override public String toString() { - return "SegmentTransactionalInsertAction{" + + return "SegmentTransactionalReplaceAction{" + "segmentsToBeOverwritten=" + SegmentUtils.commaSeparatedIdentifiers(segmentsToBeOverwritten) + ", segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + ", segmentsToBeDropped=" + SegmentUtils.commaSeparatedIdentifiers(segmentsToBeDropped) + diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java index 35653aa2303f..291b0200da04 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java @@ -33,9 +33,11 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.MarkSegmentsAsUnusedAction; import org.apache.druid.indexing.common.actions.RetrieveUnusedSegmentsAction; +import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.SegmentNukeAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskLocks; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; @@ -200,6 +202,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception if (nextBatchSize <= 0) { break; } + unusedSegments = toolbox .getTaskActionClient() .submit(new RetrieveUnusedSegmentsAction(getDataSource(), getInterval(), nextBatchSize)); @@ -220,7 +223,22 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception // abandoned. toolbox.getTaskActionClient().submit(new SegmentNukeAction(new HashSet<>(unusedSegments))); - toolbox.getDataSegmentKiller().kill(unusedSegments); + + // Fetch the load specs of all segments overlapping with the given interval + final Set> usedSegmentLoadSpecs = toolbox + .getTaskActionClient() + .submit(new RetrieveUsedSegmentsAction(getDataSource(), getInterval(), null, Segments.INCLUDING_OVERSHADOWED)) + .stream() + .map(DataSegment::getLoadSpec) + .collect(Collectors.toSet()); + + // Kill segments from the deep storage only if their load specs are not being used by any used segments + final List segmentsToBeKilled = unusedSegments + .stream() + .filter(unusedSegment -> !usedSegmentLoadSpecs.contains(unusedSegment.getLoadSpec())) + .collect(Collectors.toList()); + + toolbox.getDataSegmentKiller().kill(segmentsToBeKilled); numBatchesProcessed++; numSegmentsKilled += unusedSegments.size(); diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 5ee437914af8..9166fe926d9e 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -110,6 +110,8 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor private final MetadataStorageTablesConfig dbTables; private final SQLMetadataConnector connector; + private final String insertSegmentQuery; + @Inject public IndexerSQLMetadataStorageCoordinator( ObjectMapper jsonMapper, @@ -120,6 +122,12 @@ public IndexerSQLMetadataStorageCoordinator( this.jsonMapper = jsonMapper; this.dbTables = dbTables; this.connector = connector; + this.insertSegmentQuery = StringUtils.format( + "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload, used_flag_last_updated) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload, :used_flag_last_updated)", + dbTables.getSegmentsTable(), + connector.getQuoteString() + ); } @LifecycleStart @@ -1732,18 +1740,11 @@ private Set announceHistoricalSegmentBatch( MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE ); - PreparedBatch preparedBatch = handle.prepareBatch( - StringUtils.format( - "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload, used_flag_last_updated) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload, :used_flag_last_updated)", - dbTables.getSegmentsTable(), - connector.getQuoteString() - ) - ); + PreparedBatch preparedBatch = handle.prepareBatch(insertSegmentQuery); for (List partition : partitionedSegments) { for (DataSegment segment : partition) { - String now = DateTimes.nowUtc().toString(); + final String now = DateTimes.nowUtc().toString(); preparedBatch.add() .bind("id", segment.getId().toString()) .bind("dataSource", segment.getDataSource()) @@ -1805,26 +1806,21 @@ private Set commitReplaceSegmentBatch( MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE ); - PreparedBatch preparedBatch = handle.prepareBatch( - StringUtils.format( - "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", - dbTables.getSegmentsTable(), - connector.getQuoteString() - ) - ); + PreparedBatch preparedBatch = handle.prepareBatch(insertSegmentQuery); for (List partition : partitionedSegments) { for (DataSegment segment : partition) { + final String now = DateTimes.nowUtc().toString(); preparedBatch.add() .bind("id", segment.getId().toString()) .bind("dataSource", segment.getDataSource()) - .bind("created_date", DateTimes.nowUtc().toString()) + .bind("created_date", now) .bind("start", segment.getInterval().getStart().toString()) .bind("end", segment.getInterval().getEnd().toString()) .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) .bind("version", segment.getVersion()) .bind("used", usedSegments.contains(segment)) - .bind("payload", jsonMapper.writeValueAsBytes(segment)); + .bind("payload", jsonMapper.writeValueAsBytes(segment)) + .bind("used_flag_last_updated", now); } final int[] affectedInsertRows = preparedBatch.execute(); @@ -1843,14 +1839,6 @@ private Set commitReplaceSegmentBatch( } } - PreparedBatch appendBatch = handle.prepareBatch( - StringUtils.format( - "INSERT INTO %1$s (id, dataSource, start, %2$send%2$s, segment_id, lock_version) " - + "VALUES (:id, :dataSource, :start, :end, :segment_id, :lock_version)", - dbTables.getSegmentVersionsTable(), - connector.getQuoteString() - ) - ); Map segmentsToBeForwarded = getAppendedSegmentIds( handle, segments.iterator().next().getDataSource(), @@ -1895,16 +1883,18 @@ private Set commitReplaceSegmentBatch( oldSegment.getBinaryVersion(), oldSegment.getSize() ); + final String now = DateTimes.nowUtc().toString(); preparedBatch.add() .bind("id", newSegment.getId().toString()) .bind("dataSource", newSegment.getDataSource()) - .bind("created_date", DateTimes.nowUtc().toString()) + .bind("created_date", now) .bind("start", newSegment.getInterval().getStart().toString()) .bind("end", newSegment.getInterval().getEnd().toString()) .bind("partitioned", (newSegment.getShardSpec() instanceof NoneShardSpec) ? false : true) .bind("version", newSegment.getVersion()) .bind("used", true) - .bind("payload", jsonMapper.writeValueAsBytes(newSegment)); + .bind("payload", jsonMapper.writeValueAsBytes(newSegment)) + .bind("used_flag_last_updated", now); } final int[] affectedInsertRows = preparedBatch.execute(); @@ -1949,26 +1939,21 @@ private Set commitAppendSegmentBatch( MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE ); - PreparedBatch preparedBatch = handle.prepareBatch( - StringUtils.format( - "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", - dbTables.getSegmentsTable(), - connector.getQuoteString() - ) - ); + PreparedBatch preparedBatch = handle.prepareBatch(insertSegmentQuery); for (List partition : partitionedSegments) { for (DataSegment segment : partition) { + final String now = DateTimes.nowUtc().toString(); preparedBatch.add() .bind("id", segment.getId().toString()) .bind("dataSource", segment.getDataSource()) - .bind("created_date", DateTimes.nowUtc().toString()) + .bind("created_date", now) .bind("start", segment.getInterval().getStart().toString()) .bind("end", segment.getInterval().getEnd().toString()) .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) .bind("version", segment.getVersion()) .bind("used", usedSegments.contains(segment)) - .bind("payload", jsonMapper.writeValueAsBytes(segment)); + .bind("payload", jsonMapper.writeValueAsBytes(segment)) + .bind("used_flag_last_updated", now); } final int[] affectedInsertRows = preparedBatch.execute(); @@ -2064,26 +2049,21 @@ private Set commitSegmentBatch( MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE ); - PreparedBatch preparedBatch = handle.prepareBatch( - StringUtils.format( - "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", - dbTables.getSegmentsTable(), - connector.getQuoteString() - ) - ); + PreparedBatch preparedBatch = handle.prepareBatch(StringUtils.format(insertSegmentQuery)); for (List partition : partitionedSegments) { for (DataSegment segment : partition) { + final String now = DateTimes.nowUtc().toString(); preparedBatch.add() .bind("id", segment.getId().toString()) .bind("dataSource", segment.getDataSource()) - .bind("created_date", DateTimes.nowUtc().toString()) + .bind("created_date", now) .bind("start", segment.getInterval().getStart().toString()) .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) + .bind("partitioned", !(segment.getShardSpec() instanceof NoneShardSpec)) .bind("version", segment.getVersion()) .bind("used", usedSegments.contains(segment)) - .bind("payload", jsonMapper.writeValueAsBytes(segment)); + .bind("payload", jsonMapper.writeValueAsBytes(segment)) + .bind("used_flag_last_updated", now); } final int[] affectedInsertRows = preparedBatch.execute(); @@ -2197,7 +2177,7 @@ private Set commitSegmentBatch( .bind("created_date", DateTimes.nowUtc().toString()) .bind("start", newSegment.getInterval().getStart().toString()) .bind("end", newSegment.getInterval().getEnd().toString()) - .bind("partitioned", (newSegment.getShardSpec() instanceof NoneShardSpec) ? false : true) + .bind("partitioned", !(newSegment.getShardSpec() instanceof NoneShardSpec)) .bind("version", newSegment.getVersion()) .bind("used", true) .bind("payload", jsonMapper.writeValueAsBytes(newSegment)); From 311d0ca9479c995320a72da7b2f8c917275ce359 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 21 Aug 2023 12:27:09 +0530 Subject: [PATCH 12/43] Clean up append and replace actions --- .../SegmentTransactionalAppendAction.java | 57 +-- .../SegmentTransactionalInsertAction.java | 2 +- .../SegmentTransactionalReplaceAction.java | 46 +-- .../indexing/common/actions/TaskLocks.java | 57 ++- .../indexing/common/task/IndexTaskUtils.java | 11 + ...penderatorDriverRealtimeIndexTaskTest.java | 4 +- .../SinglePhaseParallelIndexingTest.java | 1 - ...TestIndexerMetadataStorageCoordinator.java | 5 +- .../indexing/overlord/DataSourceMetadata.java | 2 +- .../IndexerMetadataStorageCoordinator.java | 7 +- .../IndexerSQLMetadataStorageCoordinator.java | 368 ++++-------------- .../druid/metadata/SQLMetadataConnector.java | 2 +- ...exerSQLMetadataStorageCoordinatorTest.java | 32 +- 13 files changed, 167 insertions(+), 427 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java index e0d155408dc0..95f3a96aca8e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableSet; -import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; @@ -31,14 +30,10 @@ import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.TaskLockInfo; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; -import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; -import org.joda.time.Interval; import javax.annotation.Nullable; -import java.util.HashMap; -import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -111,42 +106,22 @@ public TypeReference getReturnTypeReference() @Override public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) { - final SegmentPublishResult retVal; - - final Set allSegments = new HashSet<>(segments); - - String datasource = task.getDataSource(); - Map replaceLocks = new HashMap<>(); - for (TaskLock lock : TaskLocks.findReplaceLocksForSegments(datasource, toolbox.getTaskLockbox(), segments)) { - replaceLocks.put(lock.getInterval(), lock); - } - Map appendSegmentLockMap = new HashMap<>(); - Set taskLockInfos = new HashSet<>(); - for (TaskLock taskLock : replaceLocks.values()) { - taskLockInfos.add(getTaskLockInfo(taskLock)); - } - - for (DataSegment segment : segments) { - Interval interval = segment.getInterval(); - for (Interval key : replaceLocks.keySet()) { - if (key.contains(interval)) { - appendSegmentLockMap.put(segment, getTaskLockInfo(replaceLocks.get(key))); - } - } - } + final String datasource = task.getDataSource(); + final Map segmentToReplaceLock + = TaskLocks.findReplaceLocksCoveringSegments(datasource, toolbox.getTaskLockbox(), segments); + final SegmentPublishResult retVal; try { retVal = toolbox.getTaskLockbox().doInCriticalSection( task, - allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), + segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), CriticalAction.builder() .onValidLocks( () -> toolbox.getIndexerMetadataStorageCoordinator().commitAppendSegments( segments, startMetadata, endMetadata, - appendSegmentLockMap, - taskLockInfos + segmentToReplaceLock ) ) .onInvalidLocks( @@ -168,29 +143,17 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) if (retVal.isSuccess()) { toolbox.getEmitter().emit(metricBuilder.build("segment/txn/success", 1)); + for (DataSegment segment : retVal.getSegments()) { + IndexTaskUtils.setSegmentDimensions(metricBuilder, segment); + toolbox.getEmitter().emit(metricBuilder.build("segment/added/bytes", segment.getSize())); + } } else { toolbox.getEmitter().emit(metricBuilder.build("segment/txn/failure", 1)); } - // getSegments() should return an empty set if announceHistoricalSegments() failed - for (DataSegment segment : retVal.getSegments()) { - metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); - metricBuilder.setDimension( - DruidMetrics.PARTITIONING_TYPE, - segment.getShardSpec() == null ? null : segment.getShardSpec().getType() - ); - toolbox.getEmitter().emit(metricBuilder.build("segment/added/bytes", segment.getSize())); - } - return retVal; } - - private TaskLockInfo getTaskLockInfo(TaskLock taskLock) - { - return new TaskLockInfo(taskLock.getInterval(), taskLock.getVersion()); - } - @Override public boolean isAudited() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java index 5acc048fdfe0..270154ed0ba0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java @@ -222,7 +222,7 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), CriticalAction.builder() .onValidLocks( - () -> toolbox.getIndexerMetadataStorageCoordinator().commitSegments( + () -> toolbox.getIndexerMetadataStorageCoordinator().commitSegmentsAndMetadata( segments, segmentsToBeDropped, startMetadata, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index 4dd3ea7a389a..255628e26225 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableSet; -import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; @@ -33,10 +32,8 @@ import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; -import org.joda.time.Interval; import javax.annotation.Nullable; -import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; @@ -119,32 +116,21 @@ public TypeReference getReturnTypeReference() @Override public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) { - final SegmentPublishResult retVal; - - final Set allSegments = new HashSet<>(segments); - - TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), allSegments); - String datasource = task.getDataSource(); - Map replaceLocks = new HashMap<>(); - for (TaskLock lock : TaskLocks.findReplaceLocksForSegments(datasource, toolbox.getTaskLockbox(), segments)) { - replaceLocks.put(lock.getInterval(), lock); - } - Set taskLockInfos = new HashSet<>(); - for (TaskLock taskLock : replaceLocks.values()) { - taskLockInfos.add(getTaskLockInfo(taskLock)); - } + final Map segmentToReplaceLock + = TaskLocks.findReplaceLocksCoveringSegments(datasource, toolbox.getTaskLockbox(), segments); + final SegmentPublishResult retVal; try { retVal = toolbox.getTaskLockbox().doInCriticalSection( task, - allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), + segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), CriticalAction.builder() .onValidLocks( () -> toolbox.getIndexerMetadataStorageCoordinator().commitReplaceSegments( segments, segmentsToBeDropped, - taskLockInfos + new HashSet<>(segmentToReplaceLock.values()) ) ) .onInvalidLocks( @@ -166,28 +152,20 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) if (retVal.isSuccess()) { toolbox.getEmitter().emit(metricBuilder.build("segment/txn/success", 1)); + + for (DataSegment segment : retVal.getSegments()) { + final String partitionType = segment.getShardSpec() == null ? null : segment.getShardSpec().getType(); + metricBuilder.setDimension(DruidMetrics.PARTITIONING_TYPE, partitionType); + metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); + toolbox.getEmitter().emit(metricBuilder.build("segment/added/bytes", segment.getSize())); + } } else { toolbox.getEmitter().emit(metricBuilder.build("segment/txn/failure", 1)); } - // getSegments() should return an empty set if announceHistoricalSegments() failed - for (DataSegment segment : retVal.getSegments()) { - metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); - metricBuilder.setDimension( - DruidMetrics.PARTITIONING_TYPE, - segment.getShardSpec() == null ? null : segment.getShardSpec().getType() - ); - toolbox.getEmitter().emit(metricBuilder.build("segment/added/bytes", segment.getSize())); - } - return retVal; } - private TaskLockInfo getTaskLockInfo(TaskLock taskLock) - { - return new TaskLockInfo(taskLock.getInterval(), taskLock.getVersion()); - } - @Override public boolean isAudited() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java index 400d0f38fdc3..3b4fa067bc96 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java @@ -26,20 +26,24 @@ import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TimeChunkLock; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.TaskLockInfo; import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.java.util.common.ISE; import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; +import org.joda.time.Interval; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Map.Entry; import java.util.NavigableMap; import java.util.Set; import java.util.TreeMap; +import java.util.stream.Collectors; public class TaskLocks { @@ -114,26 +118,49 @@ public static boolean isLockCoversSegments( ); } - public static Set findReplaceLocksForSegments( + /** + * Finds locks of type {@link TaskLockType#REPLACE} for each of the given segments + * that have an interval completely covering the interval of the respective segments. + * + * @return Map from segment to REPLACE lock that completely covers it. The map + * does not contain an entry for segments that have no covering REPLACE lock. + */ + public static Map findReplaceLocksCoveringSegments( final String datasource, final TaskLockbox taskLockbox, - final Collection segments + final Set segments ) { - final Set found = new HashSet<>(); - final Set locks = taskLockbox.getAllReplaceLocksForDatasource(datasource); - segments.forEach(segment -> { - locks.forEach(lock -> { - if (lock.getGranularity() == LockGranularity.TIME_CHUNK) { - final TimeChunkLock timeChunkLock = (TimeChunkLock) lock; - if (timeChunkLock.getInterval().contains(segment.getInterval()) - && timeChunkLock.getDataSource().equals(segment.getDataSource())) { - found.add(lock); - } + // Identify unique segment intervals + final Map> intervalToSegments = new HashMap<>(); + segments.forEach( + segment -> intervalToSegments.computeIfAbsent( + segment.getInterval(), interval -> new ArrayList<>() + ).add(segment) + ); + + final Set replaceLocks = taskLockbox.getAllReplaceLocksForDatasource(datasource).stream() + .map(TaskLocks::toLockInfo) + .collect(Collectors.toSet()); + + final Map segmentToReplaceLock = new HashMap<>(); + + intervalToSegments.forEach((interval, segmentsForInterval) -> { + // For each interval, find the lock that covers it, if any + for (TaskLockInfo lock : replaceLocks) { + if (lock.getInterval().contains(interval)) { + segmentsForInterval.forEach(s -> segmentToReplaceLock.put(s, lock)); + return; } - }); + } }); - return found; + + return segmentToReplaceLock; + } + + public static TaskLockInfo toLockInfo(TaskLock taskLock) + { + return new TaskLockInfo(taskLock.getInterval(), taskLock.getVersion()); } public static List findLocksForSegments( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java index cd7a52f77284..20f7584c8eb2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java @@ -33,6 +33,7 @@ import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.server.security.ResourceType; +import org.apache.druid.timeline.DataSegment; import org.apache.druid.utils.CircularBuffer; import org.joda.time.DateTime; @@ -141,4 +142,14 @@ public static void setTaskStatusDimensions( metricBuilder.setDimension(DruidMetrics.TASK_ID, taskStatus.getId()); metricBuilder.setDimension(DruidMetrics.TASK_STATUS, taskStatus.getStatusCode().toString()); } + + public static void setSegmentDimensions( + ServiceMetricEvent.Builder metricBuilder, + DataSegment segment + ) + { + final String partitionType = segment.getShardSpec() == null ? null : segment.getShardSpec().getType(); + metricBuilder.setDimension(DruidMetrics.PARTITIONING_TYPE, partitionType); + metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index db94038e07a0..eaeb74866fba 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -1523,14 +1523,14 @@ public Set commitSegments(Set segments) throws IOExcep } @Override - public SegmentPublishResult commitSegments( + public SegmentPublishResult commitSegmentsAndMetadata( Set segments, Set segmentsToDrop, DataSourceMetadata startMetadata, DataSourceMetadata endMetadata ) throws IOException { - SegmentPublishResult result = super.commitSegments(segments, segmentsToDrop, startMetadata, endMetadata); + SegmentPublishResult result = super.commitSegmentsAndMetadata(segments, segmentsToDrop, startMetadata, endMetadata); Assert.assertFalse( "Segment latch not initialized, did you forget to call expectPublishSegments?", diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index 950983d77b85..d22769c1f791 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -29,7 +29,6 @@ import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexing.common.LockGranularity; -import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.Tasks; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index e58df94ef862..16c775b01b89 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -173,8 +173,7 @@ public SegmentPublishResult commitAppendSegments( Set segments, DataSourceMetadata oldCommitMetadata, DataSourceMetadata newCommitMetadata, - @Nullable Map segmentLockMap, - @Nullable Set taskLockInfos + @Nullable Map segmentLockMap ) { // Don't actually compare metadata, just do it! @@ -182,7 +181,7 @@ public SegmentPublishResult commitAppendSegments( } @Override - public SegmentPublishResult commitSegments( + public SegmentPublishResult commitSegmentsAndMetadata( Set segments, Set segmentsToDrop, @Nullable DataSourceMetadata startMetadata, diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java b/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java index a6955ece8b80..26fc4cc9dbac 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java @@ -26,7 +26,7 @@ /** * Commit metadata for a dataSource. Used by - * {@link IndexerMetadataStorageCoordinator#commitSegments(Set, Set, DataSourceMetadata, DataSourceMetadata)} + * {@link IndexerMetadataStorageCoordinator#commitSegmentsAndMetadata(Set, Set, DataSourceMetadata, DataSourceMetadata)} * to provide metadata transactions for segment inserts. * * Two metadata instances can be added together, and any conflicts are resolved in favor of the right-hand side. diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index e5e09269b965..0edadd960586 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -254,7 +254,7 @@ SegmentIdWithShardSpec allocatePendingSegment( * commit metadata. * * If segmentsToDrop is not null and not empty, this insertion will be atomic with a insert-and-drop on inserting - * {@param segments} and dropping {@param segmentsToDrop} + * {@param segments} and dropping {@param segmentsToDrop}. * * @param segments set of segments to add, must all be from the same dataSource * @param segmentsToDrop set of segments to drop, must all be from the same dataSource @@ -272,7 +272,7 @@ SegmentIdWithShardSpec allocatePendingSegment( * @throws IllegalArgumentException if startMetadata and endMetadata are not either both null or both non-null * @throws RuntimeException if the state of metadata storage after this call is unknown */ - SegmentPublishResult commitSegments( + SegmentPublishResult commitSegmentsAndMetadata( Set segments, Set segmentsToDrop, @Nullable DataSourceMetadata startMetadata, @@ -283,8 +283,7 @@ SegmentPublishResult commitAppendSegments( Set segments, @Nullable DataSourceMetadata startMetadata, @Nullable DataSourceMetadata endMetadata, - @Nullable Map segmentLockMap, - @Nullable Set taskLockInfos + @Nullable Map segmentLockMap ) throws IOException; SegmentPublishResult commitReplaceSegments( diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 5ee437914af8..5e26bc33b66f 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -309,7 +309,7 @@ private Collection retrieveAllUsedSegmentsForIntervalsWithHandle( @Override public Set commitSegments(final Set segments) throws IOException { - final SegmentPublishResult result = commitSegments(segments, null, null, null); + final SegmentPublishResult result = commitSegmentsAndMetadata(segments, null, null, null); // Metadata transaction cannot fail because we are not trying to do one. if (!result.isSuccess()) { @@ -320,7 +320,7 @@ public Set commitSegments(final Set segments) throws I } @Override - public SegmentPublishResult commitSegments( + public SegmentPublishResult commitSegmentsAndMetadata( final Set segments, final Set segmentsToDrop, @Nullable final DataSourceMetadata startMetadata, @@ -459,41 +459,31 @@ public SegmentPublishResult commitReplaceSegments( try { return connector.retryTransaction( - new TransactionCallback() - { - @Override - public SegmentPublishResult inTransaction( - final Handle handle, - final TransactionStatus transactionStatus - ) throws Exception - { - // Set definitelyNotUpdated back to false upon retrying. - definitelyNotUpdated.set(false); - + (handle, transactionStatus) -> { + // Set definitelyNotUpdated back to false upon retrying. + definitelyNotUpdated.set(false); - if (segmentsToDrop != null && !segmentsToDrop.isEmpty()) { - final DataStoreMetadataUpdateResult result = dropSegmentsWithHandle( - handle, - segmentsToDrop, - dataSource - ); - if (result.isFailed()) { - // Metadata store was definitely not updated. - transactionStatus.setRollbackOnly(); - definitelyNotUpdated.set(true); + if (segmentsToDrop != null && !segmentsToDrop.isEmpty()) { + final DataStoreMetadataUpdateResult result = dropSegmentsWithHandle( + handle, + segmentsToDrop, + dataSource + ); + if (result.isFailed()) { + // Metadata store was definitely not updated. + transactionStatus.setRollbackOnly(); + definitelyNotUpdated.set(true); - if (result.canRetry()) { - throw new RetryTransactionException(result.getErrorMsg()); - } else { - throw new RuntimeException(result.getErrorMsg()); - } + if (result.canRetry()) { + throw new RetryTransactionException(result.getErrorMsg()); + } else { + throw new RuntimeException(result.getErrorMsg()); } } - - final Set inserted = commitReplaceSegmentBatch(handle, newSegments, usedSegments, taskLockInfos); - - return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); } + + final Set inserted = commitReplaceSegmentBatch(handle, newSegments, usedSegments, taskLockInfos); + return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); }, 3, getSqlMetadataMaxRetry() @@ -514,43 +504,28 @@ public SegmentPublishResult commitAppendSegments( final Set segments, @Nullable final DataSourceMetadata startMetadata, @Nullable DataSourceMetadata endMetadata, - @Nullable Map segmentLockMap, - @Nullable Set taskLockInfos + @Nullable Map segmentLockMap ) { if (segments.isEmpty()) { - throw new IllegalArgumentException("segment set must not be empty"); + throw new IllegalArgumentException("No segments to append"); } final String dataSource = segments.iterator().next().getDataSource(); for (DataSegment segment : segments) { if (!dataSource.equals(segment.getDataSource())) { - throw new IllegalArgumentException("segments must all be from the same dataSource"); + throw new IllegalArgumentException("All segments to append must belong to the same dataSource"); } } if ((startMetadata == null && endMetadata != null) || (startMetadata != null && endMetadata == null)) { - throw new IllegalArgumentException("start/end metadata pair must be either null or non-null"); + throw new IllegalArgumentException("Start and end metadata must either be both null or both non-null"); } // Find which segments are used (i.e. not overshadowed). Set newSegments = new HashSet<>(segments); final Map> segmentToNewMetadataMap = connector.retryTransaction( - new TransactionCallback>>() - { - @Override - public Map> inTransaction( - final Handle handle, - final TransactionStatus transactionStatus - ) throws Exception - { - return allocateNewSegmentIds( - handle, - dataSource, - segments - ); - } - }, + (handle, transactionStatus) -> allocateNewSegmentIds(handle, dataSource, segments), 0, SQLMetadataConnector.DEFAULT_MAX_TRIES ); @@ -572,49 +547,36 @@ public Map> inTransaction( } Set usedSegments = new HashSet<>(newSegments); - - final AtomicBoolean definitelyNotUpdated = new AtomicBoolean(false); - try { return connector.retryTransaction( - new TransactionCallback() - { - @Override - public SegmentPublishResult inTransaction( - final Handle handle, - final TransactionStatus transactionStatus - ) throws Exception - { - // Set definitelyNotUpdated back to false upon retrying. - definitelyNotUpdated.set(false); + (handle, transactionStatus) -> { + // Set definitelyNotUpdated back to false upon retrying. + definitelyNotUpdated.set(false); - if (startMetadata != null) { - final DataStoreMetadataUpdateResult result = updateDataSourceMetadataWithHandle( - handle, - dataSource, - startMetadata, - endMetadata - ); + if (startMetadata != null) { + final DataStoreMetadataUpdateResult result = updateDataSourceMetadataWithHandle( + handle, + dataSource, + startMetadata, + endMetadata + ); - if (result.isFailed()) { - // Metadata was definitely not updated. - transactionStatus.setRollbackOnly(); - definitelyNotUpdated.set(true); + if (result.isFailed()) { + // Metadata was definitely not updated. + transactionStatus.setRollbackOnly(); + definitelyNotUpdated.set(true); - if (result.canRetry()) { - throw new RetryTransactionException(result.getErrorMsg()); - } else { - throw new RuntimeException(result.getErrorMsg()); - } + if (result.canRetry()) { + throw new RetryTransactionException(result.getErrorMsg()); + } else { + throw new RuntimeException(result.getErrorMsg()); } } - - - final Set inserted = commitAppendSegmentBatch(handle, newSegments, usedSegments, segmentLockMap); - - return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); } + + final Set inserted = commitAppendSegmentBatch(handle, newSegments, usedSegments, segmentLockMap); + return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); }, 3, getSqlMetadataMaxRetry() @@ -1299,7 +1261,7 @@ private Map> allocateNewSegmentIds( } for (DataSegment segment : intervalToSegments.get(interval)) { SegmentCreateRequest request = new SegmentCreateRequest( - segment.getId().toString() + version, + segment.getId() + version, null, version, NumberedPartialShardSpec.instance() @@ -1867,7 +1829,7 @@ private Set commitReplaceSegmentBatch( for (Map.Entry entry : batch) { batchMap.put(entry.getKey(), entry.getValue()); } - List oldSegments = retrieveSegments(handle, batchMap.keySet()); + List oldSegments = retrieveSegmentsById(handle, batchMap.keySet()); for (DataSegment oldSegment : oldSegments) { Interval newInterval = oldSegment.getInterval(); for (DataSegment segment : segments) { @@ -2007,122 +1969,7 @@ private Set commitAppendSegmentBatch( DataSegment segment = entry.getKey(); TaskLockInfo lock = entry.getValue(); appendBatch.add() - .bind("id", segment.getId().toString() + ":" + lock.hashCode()) - .bind("dataSource", segment.getDataSource()) - .bind("start", lock.getInterval().getStartMillis()) - .bind("end", lock.getInterval().getEndMillis()) - .bind("segment_id", segment.getId().toString()) - .bind("lock_version", lock.getVersion()); - } - final int[] affectedAppendRows = appendBatch.execute(); - final boolean succeeded = Arrays.stream(affectedAppendRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); - if (!succeeded) { - final List failedToForward = IntStream.range(0, partition.size()) - .filter(i -> affectedAppendRows[i] != 1) - .mapToObj(partition::get) - .map(x -> x.getKey()) - .collect(Collectors.toList()); - throw new ISE( - "Failed to forward appended segments to DB: %s", - SegmentUtils.commaSeparatedIdentifiers(failedToForward) - ); - } - } - } - catch (Exception e) { - log.errorSegments(segments, "Exception inserting segment metadata"); - throw e; - } - - return toInsertSegments; - } - - private Set commitSegmentBatch( - final Handle handle, - final Set segments, - final Set usedSegments, - @Nullable Map appendSegmentLockMap, - @Nullable Set replaceLocks, - boolean append - ) throws IOException - { - final Set toInsertSegments = new HashSet<>(); - try { - Set existedSegments = segmentExistsBatch(handle, segments); - log.info("Found these segments already exist in DB: %s", existedSegments); - for (DataSegment segment : segments) { - if (!existedSegments.contains(segment.getId().toString())) { - toInsertSegments.add(segment); - } - } - - // SELECT -> INSERT can fail due to races; callers must be prepared to retry. - // Avoiding ON DUPLICATE KEY since it's not portable. - // Avoiding try/catch since it may cause inadvertent transaction-splitting. - final List> partitionedSegments = Lists.partition( - new ArrayList<>(toInsertSegments), - MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE - ); - - PreparedBatch preparedBatch = handle.prepareBatch( - StringUtils.format( - "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload) " - + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", - dbTables.getSegmentsTable(), - connector.getQuoteString() - ) - ); - for (List partition : partitionedSegments) { - for (DataSegment segment : partition) { - preparedBatch.add() - .bind("id", segment.getId().toString()) - .bind("dataSource", segment.getDataSource()) - .bind("created_date", DateTimes.nowUtc().toString()) - .bind("start", segment.getInterval().getStart().toString()) - .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) - .bind("version", segment.getVersion()) - .bind("used", usedSegments.contains(segment)) - .bind("payload", jsonMapper.writeValueAsBytes(segment)); - } - final int[] affectedInsertRows = preparedBatch.execute(); - - final boolean succeeded = Arrays.stream(affectedInsertRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); - if (succeeded) { - log.infoSegments(partition, "Published segments to DB"); - } else { - final List failedToPublish = IntStream.range(0, partition.size()) - .filter(i -> affectedInsertRows[i] != 1) - .mapToObj(partition::get) - .collect(Collectors.toList()); - throw new ISE( - "Failed to publish segments to DB: %s", - SegmentUtils.commaSeparatedIdentifiers(failedToPublish) - ); - } - } - - PreparedBatch appendBatch = handle.prepareBatch( - StringUtils.format( - "INSERT INTO %1$s (id, dataSource, start, %2$send%2$s, segment_id, lock_version) " - + "VALUES (:id, :dataSource, :start, :end, :segment_id, :lock_version)", - dbTables.getSegmentVersionsTable(), - connector.getQuoteString() - ) - ); - if (appendSegmentLockMap == null) { - appendSegmentLockMap = new HashMap<>(); - } - final List>> appendSegmentPartitions = Lists.partition( - new ArrayList<>(appendSegmentLockMap.entrySet()), - MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE - ); - for (List> partition : appendSegmentPartitions) { - for (Map.Entry entry : partition) { - DataSegment segment = entry.getKey(); - TaskLockInfo lock = entry.getValue(); - appendBatch.add() - .bind("id", segment.getId().toString() + ":" + lock.hashCode()) + .bind("id", segment.getId() + ":" + lock.hashCode()) .bind("dataSource", segment.getDataSource()) .bind("start", lock.getInterval().getStartMillis()) .bind("end", lock.getInterval().getEndMillis()) @@ -2143,74 +1990,6 @@ private Set commitSegmentBatch( ); } } - - Map segmentsToBeForwarded = new HashMap<>(); - if (!append) { - segmentsToBeForwarded = getAppendedSegmentIds( - handle, - segments.iterator().next().getDataSource(), - replaceLocks - ); - } - final int numCorePartitions = segments.size(); - int partitionNum = segments.size(); - final List>> forwardSegmentsBatch = Lists.partition( - new ArrayList<>(segmentsToBeForwarded.entrySet()), - MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE - ); - for (List> batch : forwardSegmentsBatch) { - Map batchMap = new HashMap<>(); - for (Map.Entry entry : batch) { - batchMap.put(entry.getKey(), entry.getValue()); - } - List oldSegments = retrieveSegments(handle, batchMap.keySet()); - for (DataSegment oldSegment : oldSegments) { - Interval newInterval = oldSegment.getInterval(); - for (DataSegment segment : segments) { - if (segment.getInterval().overlaps(newInterval)) { - if (segment.getInterval().contains(newInterval)) { - newInterval = segment.getInterval(); - } else { - throw new ISE("Incompatible segment intervals for commit: [%s] and [%s].", - newInterval, - segment.getInterval() - ); - } - } - } - TaskLockInfo lock = batchMap.get(oldSegment.getId().toString()); - ShardSpec shardSpec = new NumberedShardSpec(partitionNum++, numCorePartitions); - DataSegment newSegment = new DataSegment( - oldSegment.getDataSource(), - newInterval, - lock.getVersion(), - oldSegment.getLoadSpec(), - oldSegment.getDimensions(), - oldSegment.getMetrics(), - shardSpec, - oldSegment.getBinaryVersion(), - oldSegment.getSize() - ); - preparedBatch.add() - .bind("id", newSegment.getId().toString()) - .bind("dataSource", newSegment.getDataSource()) - .bind("created_date", DateTimes.nowUtc().toString()) - .bind("start", newSegment.getInterval().getStart().toString()) - .bind("end", newSegment.getInterval().getEnd().toString()) - .bind("partitioned", (newSegment.getShardSpec() instanceof NoneShardSpec) ? false : true) - .bind("version", newSegment.getVersion()) - .bind("used", true) - .bind("payload", jsonMapper.writeValueAsBytes(newSegment)); - } - final int[] affectedInsertRows = preparedBatch.execute(); - - final boolean succeeded = Arrays.stream(affectedInsertRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); - if (succeeded) { - log.info("Published segments with updated metadata to DB"); - } else { - throw new ISE("Failed to update segment metadatas in DB"); - } - } } catch (Exception e) { log.errorSegments(segments, "Exception inserting segment metadata"); @@ -2220,37 +1999,22 @@ private Set commitSegmentBatch( return toInsertSegments; } - private List retrieveSegments(final Handle handle, final Set segmentIds) + private List retrieveSegmentsById(Handle handle, Set segmentIds) { - final StringBuilder sb = new StringBuilder(); - sb.append("SELECT payload FROM %s WHERE id in ("); - - List segmentIdList = new ArrayList<>(segmentIds); - int n = segmentIdList.size(); - for (int i = 0; i < n; i++) { - sb.append("'"); - sb.append(segmentIdList.get(i)); - sb.append("'"); - if (i < n - 1) { - sb.append(", "); - } - } - - sb.append(")"); - - final Query> sql = handle - .createQuery(StringUtils.format(sb.toString(), dbTables.getSegmentsTable())) - .setFetchSize(connector.getStreamingFetchSize()); + final String segmentIdCsv = segmentIds.stream().map(id -> "'" + id + "'") + .collect(Collectors.joining(",")); + final Query> query = handle.createQuery( + StringUtils.format( + "SELECT payload FROM %s WHERE id in (%s)", + segmentIdCsv, dbTables.getSegmentsTable() + ) + ).setFetchSize(connector.getStreamingFetchSize()); - final ResultIterator resultIterator = - sql.map((index, r, ctx) -> JacksonUtils.readValue(jsonMapper, r.getBytes(1), DataSegment.class)) - .iterator(); + ResultIterator resultIterator = query.map( + (index, r, ctx) -> JacksonUtils.readValue(jsonMapper, r.getBytes(1), DataSegment.class) + ).iterator(); - List retVal = new ArrayList<>(); - while (resultIterator.hasNext()) { - retVal.add(resultIterator.next()); - } - return retVal; + return Lists.newArrayList(resultIterator); } private Map getAppendedSegmentIds( @@ -2383,7 +2147,7 @@ private Set segmentExistsBatch(final Handle handle, final Set dropSegments = ImmutableSet.of(existingSegment1, existingSegment2, dataSegmentBar); - final SegmentPublishResult result1 = coordinator.commitSegments( + final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( SEGMENTS, dropSegments, null, @@ -750,7 +750,7 @@ public void testTransactionalAnnounceSucceedWithSegmentDrop() throws IOException retrieveUsedSegmentIds() ); - final SegmentPublishResult result1 = coordinator.commitSegments( + final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( SEGMENTS, ImmutableSet.of(existingSegment1, existingSegment2), null, @@ -788,7 +788,7 @@ public void testTransactionalAnnounceFailSegmentDropFailWithRetry() throws IOExc ); Set dropSegments = ImmutableSet.of(existingSegment1, defaultSegment4); - final SegmentPublishResult result1 = coordinator.commitSegments( + final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( SEGMENTS, dropSegments, null, @@ -809,7 +809,7 @@ public void testTransactionalAnnounceFailSegmentDropFailWithRetry() throws IOExc @Test public void testTransactionalAnnounceFailDbNotNullWantNull() throws IOException { - final SegmentPublishResult result1 = coordinator.commitSegments( + final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), ImmutableSet.of(), new ObjectMetadata(null), @@ -817,7 +817,7 @@ public void testTransactionalAnnounceFailDbNotNullWantNull() throws IOException ); Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(defaultSegment)), result1); - final SegmentPublishResult result2 = coordinator.commitSegments( + final SegmentPublishResult result2 = coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment2), ImmutableSet.of(), new ObjectMetadata(null), @@ -835,7 +835,7 @@ public void testTransactionalAnnounceFailDbNotNullWantNull() throws IOException @Test public void testTransactionalAnnounceFailDbNotNullWantDifferent() throws IOException { - final SegmentPublishResult result1 = coordinator.commitSegments( + final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), ImmutableSet.of(), new ObjectMetadata(null), @@ -843,7 +843,7 @@ public void testTransactionalAnnounceFailDbNotNullWantDifferent() throws IOExcep ); Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(defaultSegment)), result1); - final SegmentPublishResult result2 = coordinator.commitSegments( + final SegmentPublishResult result2 = coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment2), ImmutableSet.of(), new ObjectMetadata(ImmutableMap.of("foo", "qux")), @@ -1391,7 +1391,7 @@ public void testSecondHalfEternityMultipleSegmentWithStringComparison() throws I @Test public void testDeleteDataSourceMetadata() throws IOException { - coordinator.commitSegments( + coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), ImmutableSet.of(), new ObjectMetadata(null), @@ -2347,7 +2347,7 @@ public void testDropSegmentsWithHandleForSegmentThatDoesNotExist() @Test public void testRemoveDataSourceMetadataOlderThanDatasourceActiveShouldNotBeDeleted() throws Exception { - coordinator.commitSegments( + coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), ImmutableSet.of(), new ObjectMetadata(null), @@ -2376,7 +2376,7 @@ public void testRemoveDataSourceMetadataOlderThanDatasourceActiveShouldNotBeDele @Test public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveAndOlderThanTimeShouldBeDeleted() throws Exception { - coordinator.commitSegments( + coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), ImmutableSet.of(), new ObjectMetadata(null), @@ -2402,7 +2402,7 @@ public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveAndOlderThan public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveButNotOlderThanTimeShouldNotBeDeleted() throws Exception { - coordinator.commitSegments( + coordinator.commitSegmentsAndMetadata( ImmutableSet.of(defaultSegment), ImmutableSet.of(), new ObjectMetadata(null), From a2732ca6bfba92cf52217709a1bf8d5ecdf86484 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 21 Aug 2023 12:41:52 +0530 Subject: [PATCH 13/43] Fix SQL, remove forbidden APIs --- .../indexing/common/task/batch/parallel/AppendTask.java | 7 +------ .../metadata/IndexerSQLMetadataStorageCoordinator.java | 2 +- 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java index 14a4d815fce2..a5e1cebf326b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java @@ -175,7 +175,7 @@ public void cleanUp(TaskToolbox toolbox, TaskStatus taskStatus) throws Exception public SegmentIdWithShardSpec allocateOrGetSegmentForTimestamp(String timestamp) { - final DateTime time = DateTime.parse(timestamp); + final DateTime time = DateTimes.of(timestamp); for (SegmentIdWithShardSpec pendingSegment : pendingSegments) { if (pendingSegment.getInterval().contains(time)) { return pendingSegment; @@ -184,11 +184,6 @@ public SegmentIdWithShardSpec allocateOrGetSegmentForTimestamp(String timestamp) return allocateNewSegmentForDate(time); } - public SegmentIdWithShardSpec allocateNewSegmentForTimestamp(String timestamp) - { - return allocateNewSegmentForDate(DateTime.parse(timestamp)); - } - private SegmentIdWithShardSpec allocateNewSegmentForDate(DateTime time) { try { diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index c6327bf35388..66d27a1fabcd 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1990,7 +1990,7 @@ private List retrieveSegmentsById(Handle handle, Set segmen final Query> query = handle.createQuery( StringUtils.format( "SELECT payload FROM %s WHERE id in (%s)", - segmentIdCsv, dbTables.getSegmentsTable() + dbTables.getSegmentsTable(), segmentIdCsv ) ).setFetchSize(connector.getStreamingFetchSize()); From 8f1e1652762d3e8e71f26cc72acc89b219d105a7 Mon Sep 17 00:00:00 2001 From: Amatya Date: Tue, 22 Aug 2023 12:39:12 +0530 Subject: [PATCH 14/43] Add unit tests --- .../task/batch/parallel/AppendTask.java | 2 +- .../IndexerSQLMetadataStorageCoordinator.java | 62 +-- ...exerSQLMetadataStorageCoordinatorTest.java | 360 ++++++++++++++++++ 3 files changed, 396 insertions(+), 28 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java index a5e1cebf326b..217aabdb4d2b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java @@ -94,7 +94,7 @@ public AppendTask( ) { super( - id == null ? StringUtils.format("replace_%s_%s", DateTimes.nowUtc(), UUID.randomUUID().toString()) : id, + id == null ? StringUtils.format("append_%s_%s", DateTimes.nowUtc(), UUID.randomUUID().toString()) : id, dataSource == null ? "none" : dataSource, null, IngestionMode.APPEND diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 66d27a1fabcd..b4a27820ef94 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -452,17 +452,6 @@ public SegmentPublishResult commitReplaceSegments( } } - // Find which segments are used (i.e. not overshadowed). - Set usedSegments = new HashSet<>(); - Set newSegments = new HashSet<>(segments); - List> segmentHolders = - SegmentTimeline.forSegments(segments).lookupWithIncompletePartitions(Intervals.ETERNITY); - for (TimelineObjectHolder holder : segmentHolders) { - for (PartitionChunk chunk : holder.getObject()) { - usedSegments.add(chunk.getObject()); - } - } - final AtomicBoolean definitelyNotUpdated = new AtomicBoolean(false); try { @@ -490,7 +479,7 @@ public SegmentPublishResult commitReplaceSegments( } } - final Set inserted = commitReplaceSegmentBatch(handle, newSegments, usedSegments, taskLockInfos); + final Set inserted = commitReplaceSegmentBatch(handle, segments, taskLockInfos); return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); }, 3, @@ -553,7 +542,6 @@ public SegmentPublishResult commitAppendSegments( newSegments.add(newSegment); } } - Set usedSegments = new HashSet<>(newSegments); final AtomicBoolean definitelyNotUpdated = new AtomicBoolean(false); try { @@ -583,7 +571,7 @@ public SegmentPublishResult commitAppendSegments( } } - final Set inserted = commitAppendSegmentBatch(handle, newSegments, usedSegments, segmentLockMap); + final Set inserted = commitAppendSegmentBatch(handle, newSegments, segmentLockMap); return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); }, 3, @@ -1178,7 +1166,8 @@ private void insertPendingSegmentIntoMetastore( .execute(); } - private Map> allocateNewSegmentIds( + @VisibleForTesting + Map> allocateNewSegmentIds( Handle handle, String dataSource, Set segments @@ -1190,6 +1179,7 @@ private Map> allocateNewSegmentIds( // Map from version to used committed segments Map> versionToSegments = new HashMap<>(); + Map> versionToIntervals = new HashMap<>(); Collection segmentIntervals = segments.stream() .map(DataSegment::getInterval) .collect(Collectors.toSet()); @@ -1204,6 +1194,8 @@ private Map> allocateNewSegmentIds( final DataSegment segment = iterator.next(); versionToSegments.computeIfAbsent(segment.getVersion(), v -> new HashSet<>()) .add(segment); + versionToIntervals.computeIfAbsent(segment.getVersion(), v -> new HashSet<>()) + .add(segment.getInterval()); } } @@ -1213,11 +1205,16 @@ private Map> allocateNewSegmentIds( retVal.put(segment, new HashSet<>()); } - for (String version : versionToSegments.keySet()) { + for (final String version : versionToSegments.keySet()) { Set lowerVersionSegments = new HashSet<>(); - for (DataSegment segment : segments) { + for (final DataSegment segment : segments) { if (segment.getVersion().compareTo(version) < 0) { - lowerVersionSegments.add(segment); + for (final Interval interval : versionToIntervals.get(version)) { + if (interval.overlaps(segment.getInterval())) { + lowerVersionSegments.add(segment); + break; + } + } } } @@ -1745,7 +1742,6 @@ private Set announceHistoricalSegmentBatch( private Set commitReplaceSegmentBatch( final Handle handle, final Set segments, - final Set usedSegments, @Nullable Set replaceLocks ) throws IOException { @@ -1779,7 +1775,7 @@ private Set commitReplaceSegmentBatch( .bind("end", segment.getInterval().getEnd().toString()) .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) .bind("version", segment.getVersion()) - .bind("used", usedSegments.contains(segment)) + .bind("used", true) .bind("payload", jsonMapper.writeValueAsBytes(segment)) .bind("used_status_last_updated", now); } @@ -1805,8 +1801,17 @@ private Set commitReplaceSegmentBatch( segments.iterator().next().getDataSource(), replaceLocks ); - final int numCorePartitions = segments.size(); - int partitionNum = segments.size(); + Map intervalToCorePartition = new HashMap<>(); + Map intervalToPartitionNum = new HashMap<>(); + for (DataSegment segment : toInsertSegments) { + intervalToCorePartition.put(segment.getInterval(), segment.getShardSpec().getNumCorePartitions()); + intervalToPartitionNum.putIfAbsent(segment.getInterval(), 0); + int maxPartitionNum = Integer.max( + intervalToPartitionNum.get(segment.getInterval()), + segment.getShardSpec().getPartitionNum() + ); + intervalToPartitionNum.put(segment.getInterval(), maxPartitionNum); + } final List>> forwardSegmentsBatch = Lists.partition( new ArrayList<>(segmentsToBeForwarded.entrySet()), MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE @@ -1819,7 +1824,7 @@ private Set commitReplaceSegmentBatch( List oldSegments = retrieveSegmentsById(handle, batchMap.keySet()); for (DataSegment oldSegment : oldSegments) { Interval newInterval = oldSegment.getInterval(); - for (DataSegment segment : segments) { + for (DataSegment segment : toInsertSegments) { if (segment.getInterval().overlaps(newInterval)) { if (segment.getInterval().contains(newInterval)) { newInterval = segment.getInterval(); @@ -1832,7 +1837,10 @@ private Set commitReplaceSegmentBatch( } } TaskLockInfo lock = batchMap.get(oldSegment.getId().toString()); - ShardSpec shardSpec = new NumberedShardSpec(partitionNum++, numCorePartitions); + final int partitionNum = intervalToPartitionNum.get(newInterval) + 1; + final int numCorePartitions = intervalToCorePartition.get(newInterval); + ShardSpec shardSpec = new NumberedShardSpec(partitionNum, numCorePartitions); + intervalToPartitionNum.put(newInterval, partitionNum); DataSegment newSegment = new DataSegment( oldSegment.getDataSource(), newInterval, @@ -1878,7 +1886,6 @@ private Set commitReplaceSegmentBatch( private Set commitAppendSegmentBatch( final Handle handle, final Set segments, - final Set usedSegments, @Nullable Map appendSegmentLockMap ) throws IOException { @@ -1912,7 +1919,7 @@ private Set commitAppendSegmentBatch( .bind("end", segment.getInterval().getEnd().toString()) .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) .bind("version", segment.getVersion()) - .bind("used", usedSegments.contains(segment)) + .bind("used", true) .bind("payload", jsonMapper.writeValueAsBytes(segment)) .bind("used_status_last_updated", now); } @@ -2001,7 +2008,8 @@ private List retrieveSegmentsById(Handle handle, Set segmen return Lists.newArrayList(resultIterator); } - private Map getAppendedSegmentIds( + @VisibleForTesting + Map getAppendedSegmentIds( Handle handle, String datasource, Set replaceLocks diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 5ac7d3541fb5..3a5094e8d49c 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -30,13 +30,16 @@ import org.apache.druid.indexing.overlord.SegmentCreateRequest; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.indexing.overlord.TaskLockInfo; 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.jackson.JacksonUtils; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.DimensionRangeShardSpec; import org.apache.druid.timeline.partition.HashBasedNumberedPartialShardSpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; @@ -49,6 +52,7 @@ import org.apache.druid.timeline.partition.PartialShardSpec; import org.apache.druid.timeline.partition.PartitionIds; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; +import org.apache.druid.utils.CollectionUtils; import org.assertj.core.api.Assertions; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -68,8 +72,10 @@ import java.util.Collection; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; @@ -330,6 +336,7 @@ public void setUp() derbyConnector.createDataSourceTable(); derbyConnector.createTaskTables(); derbyConnector.createSegmentTable(); + derbyConnector.createSegmentVersionTable(); derbyConnector.createPendingSegmentsTable(); metadataUpdateCounter.set(0); segmentTableDropUpdateCounter.set(0); @@ -415,6 +422,16 @@ private List retrieveUsedSegmentIds() ); } + private List retrieveUsedSegments() + { + final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); + return derbyConnector.retryWithHandle( + handle -> handle.createQuery("SELECT payload FROM " + table + " WHERE used = true ORDER BY id") + .map((index, result, context) -> JacksonUtils.readValue(mapper, result.getBytes(1), DataSegment.class)) + .list() + ); + } + private List retrieveUnusedSegmentIds() { final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); @@ -463,6 +480,349 @@ private Boolean insertUsedSegments(Set dataSegments) ); } + private Map getAppendedSegmentIds(String datasource, Set replaceLocks) + { + return derbyConnector.retryWithHandle( + handle -> { + return coordinator.getAppendedSegmentIds(handle, datasource, replaceLocks); + } + ); + } + + private Boolean insertIntoSegmentVersionsTable(Map segmentToTaskLockMap) + { + final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentVersionsTable(); + return derbyConnector.retryWithHandle( + handle -> { + PreparedBatch preparedBatch = handle.prepareBatch( + StringUtils.format( + StringUtils.format( + "INSERT INTO %1$s (id, dataSource, start, %2$send%2$s, segment_id, lock_version) " + + "VALUES (:id, :dataSource, :start, :end, :segment_id, :lock_version)", + table, + derbyConnector.getQuoteString() + ) + ) + ); + for (Map.Entry entry : segmentToTaskLockMap.entrySet()) { + final DataSegment segment = entry.getKey(); + final TaskLockInfo lock = entry.getValue(); + preparedBatch.add() + .bind("id", segment.getId().toString() + ":" + lock.hashCode()) + .bind("dataSource", segment.getDataSource()) + .bind("start", lock.getInterval().getStartMillis()) + .bind("end", lock.getInterval().getEndMillis()) + .bind("segment_id", segment.getId().toString()) + .bind("lock_version", lock.getVersion()); + } + + final int[] affectedRows = preparedBatch.execute(); + final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); + if (!succeeded) { + throw new ISE("Failed to publish segment to lock metadata mapping to DB"); + } + return true; + } + ); + } + + @Test + public void testAllocateNewSegmentIds() + { + final String v0 = "1970-01-01"; + final String v1 = "2023-01-03"; + final String v2 = "2023-02-01"; + + final Set day1 = new HashSet<>(); + for (int i = 0; i < 10; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-01/2023-01-02"), + v0, + ImmutableMap.of("path", "a-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(i), + 9, + 100 + ); + day1.add(segment); + } + final Set day2 = new HashSet<>(); + for (int i = 0; i < 10; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-02/2023-01-03"), + v0, + ImmutableMap.of("path", "b-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(i), + 9, + 100 + ); + day2.add(segment); + } + final Set day3 = new HashSet<>(); + for (int i = 0; i < 10; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-03/2023-01-04"), + v0, + ImmutableMap.of("path", "c-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(i), + 9, + 100 + ); + day3.add(segment); + } + final Set month2 = new HashSet<>(); + for (int i = 0; i < 10; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-02-01/2023-03-01"), + v0, + ImmutableMap.of("path", "x-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(i), + 9, + 100 + ); + month2.add(segment); + } + + final Set higherVersionUsedSegments = new HashSet<>(); + for (int i = 0; i < 10; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-01/2023-01-02"), + v1, + ImmutableMap.of("path", "d-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(i, 5), + 9, + 100 + ); + higherVersionUsedSegments.add(segment); + } + for (int i = 0; i < 10; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-02/2023-01-03"), + v1, + ImmutableMap.of("path", "e-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(i, 0), + 9, + 100 + ); + higherVersionUsedSegments.add(segment); + } + for (int i = 0; i < 10; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-01/2023-02-01"), + v2, + ImmutableMap.of("path", "f-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(i, 10), + 9, + 100 + ); + higherVersionUsedSegments.add(segment); + } + insertUsedSegments(higherVersionUsedSegments); + + final Set segmentsToBeProcessed = new HashSet<>(); + final Set month1 = new HashSet<>(); + month1.addAll(day1); + month1.addAll(day2); + month1.addAll(day3); + segmentsToBeProcessed.addAll(month1); + segmentsToBeProcessed.addAll(month2); + final Map> segmentToNewIds = derbyConnector.retryWithHandle( + handle -> { + return coordinator.allocateNewSegmentIds(handle, "foo", segmentsToBeProcessed); + } + ); + + for (DataSegment segment : day1) { + final Set newIds = segmentToNewIds.get(segment); + Assert.assertEquals(2, newIds.size()); + Assert.assertEquals( + ImmutableSet.of(v1, v2), + newIds.stream().map(SegmentIdWithShardSpec::getVersion).collect(Collectors.toSet()) + ); + } + for (DataSegment segment : day2) { + final Set newIds = segmentToNewIds.get(segment); + Assert.assertEquals(2, newIds.size()); + Assert.assertEquals( + ImmutableSet.of(v1, v2), + newIds.stream().map(SegmentIdWithShardSpec::getVersion).collect(Collectors.toSet()) + ); + } + for (DataSegment segment : day3) { + final Set newIds = segmentToNewIds.get(segment); + Assert.assertEquals(1, newIds.size()); + Assert.assertEquals( + ImmutableSet.of(v2), + newIds.stream().map(SegmentIdWithShardSpec::getVersion).collect(Collectors.toSet()) + ); + } + for (DataSegment segment : month2) { + Assert.assertTrue(CollectionUtils.isNullOrEmpty(segmentToNewIds.get(segment))); + } + } + + @Test + public void testCommitAppendSegments() + { + final Set allSegments = new HashSet<>(); + final Set segmentIdsToBeCarriedForward = new HashSet<>(); + final TaskLockInfo lock = new TaskLockInfo(Intervals.of("2023-01-01/2023-01-03"), "2024-01-01"); + final Map segmentLockMap = new HashMap<>(); + + for (int i = 0; i < 10; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-01/2023-01-02"), + "2023-01-01", + ImmutableMap.of("path", "a-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(i), + 9, + 100 + ); + allSegments.add(segment); + segmentIdsToBeCarriedForward.add(segment.getId().toString()); + segmentLockMap.put(segment, lock); + } + + for (int i = 0; i < 10; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-02/2023-01-03"), + "2023-01-02", + ImmutableMap.of("path", "b-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(i), + 9, + 100 + ); + allSegments.add(segment); + segmentIdsToBeCarriedForward.add(segment.getId().toString()); + segmentLockMap.put(segment, lock); + } + + for (int i = 0; i < 10; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-03/2023-01-04"), + "2023-01-03", + ImmutableMap.of("path", "c-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(i), + 9, + 100 + ); + allSegments.add(segment); + } + + coordinator.commitAppendSegments(allSegments, null, null, segmentLockMap); + + Assert.assertEquals( + allSegments.stream().map(DataSegment::getId).map(SegmentId::toString).collect(Collectors.toSet()), + ImmutableSet.copyOf(retrieveUsedSegmentIds()) + ); + + final Set replaceLocks = Collections.singleton(lock); + final Map segmentLockMetadata = getAppendedSegmentIds("foo", replaceLocks); + Assert.assertEquals(segmentIdsToBeCarriedForward, segmentLockMetadata.keySet()); + Assert.assertEquals( + lock, + Iterables.getOnlyElement(ImmutableSet.copyOf(segmentLockMetadata.values())) + ); + } + + + @Test + public void testCommitReplaceSegments() + { + final TaskLockInfo replaceLock = new TaskLockInfo(Intervals.of("2023-01-01/2023-02-01"), "2023-02-01"); + final Set segmentsAppendedWithReplaceLock = new HashSet<>(); + final Map appendedSegmentToReplaceLockMap = new HashMap<>(); + for (int i = 1; i < 9; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-0" + i + "/2023-01-0" + (i + 1)), + "2023-01-0" + i, + ImmutableMap.of("path", "a-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(0), + 9, + 100 + ); + segmentsAppendedWithReplaceLock.add(segment); + appendedSegmentToReplaceLockMap.put(segment, replaceLock); + } + insertUsedSegments(segmentsAppendedWithReplaceLock); + insertIntoSegmentVersionsTable(appendedSegmentToReplaceLockMap); + + final Set replacingSegments = new HashSet<>(); + for (int i = 1; i < 9; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-01/2023-02-01"), + "2023-02-01", + ImmutableMap.of("path", "b-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new NumberedShardSpec(i, 9), + 9, + 100 + ); + replacingSegments.add(segment); + } + + coordinator.commitReplaceSegments(replacingSegments, null, ImmutableSet.of(replaceLock)); + + Assert.assertEquals( + 2 * segmentsAppendedWithReplaceLock.size() + replacingSegments.size(), + retrieveUsedSegmentIds().size() + ); + + final Set usedSegments = new HashSet<>(retrieveUsedSegments()); + + Assert.assertTrue(usedSegments.containsAll(segmentsAppendedWithReplaceLock)); + usedSegments.removeAll(segmentsAppendedWithReplaceLock); + + Assert.assertTrue(usedSegments.containsAll(replacingSegments)); + usedSegments.removeAll(replacingSegments); + + Assert.assertEquals(segmentsAppendedWithReplaceLock.size(), usedSegments.size()); + for (DataSegment segmentReplicaWithNewVersion : usedSegments) { + boolean hasBeenCarriedForward = false; + for (DataSegment appendedSegment : segmentsAppendedWithReplaceLock) { + if (appendedSegment.getLoadSpec().equals(segmentReplicaWithNewVersion.getLoadSpec())) { + hasBeenCarriedForward = true; + break; + } + } + Assert.assertTrue(hasBeenCarriedForward); + } + } + @Test public void testSimpleAnnounce() throws IOException { From 06cf8d361c5c69b6ea515be17599412bbf0a85ac Mon Sep 17 00:00:00 2001 From: Amatya Date: Tue, 22 Aug 2023 20:39:27 +0530 Subject: [PATCH 15/43] Add comments --- .../IndexerSQLMetadataStorageCoordinator.java | 60 ++++++++++++++++++- 1 file changed, 58 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index ca29db882b17..b5784e5cf96a 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -480,6 +480,8 @@ public SegmentPublishResult commitAppendSegments( // Find which segments are used (i.e. not overshadowed). Set newSegments = new HashSet<>(segments); + // Allocate new segment ids for the same segments based on the presence of used segments with higher versions. + // It happens when replace jobs have completed with greater versions between the allocation and commit of segments. final Map> segmentToNewMetadataMap = connector.retryTransaction( (handle, transactionStatus) -> allocateNewSegmentIds(handle, dataSource, segments), 0, @@ -530,6 +532,8 @@ public SegmentPublishResult commitAppendSegments( } } + // Commit not only the segments generated by the appending job initially, + // but also those ids which need to be appended to future versions. final Set inserted = commitAppendSegmentBatch(handle, newSegments, segmentLockMap); return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); }, @@ -1125,6 +1129,14 @@ private void insertPendingSegmentIntoMetastore( .execute(); } + /** + * Find versions to which appending segments must be committed to and create appropriate segment identifiers for them + * @param handle - Handle to interact with the metadata store + * @param dataSource - datasource to which the appending job is writing + * @param segments - Set of segments generated by the append job + * @return Mapping of input segments to all the Ids with which its metadata must be committed + * @throws IOException + */ @VisibleForTesting Map> allocateNewSegmentIds( Handle handle, @@ -1136,7 +1148,7 @@ Map> allocateNewSegmentIds( return Collections.emptyMap(); } - // Map from version to used committed segments + // Store the mapping from version to segment / interval for which used segments already exist Map> versionToSegments = new HashMap<>(); Map> versionToIntervals = new HashMap<>(); Collection segmentIntervals = segments.stream() @@ -1164,6 +1176,7 @@ Map> allocateNewSegmentIds( retVal.put(segment, new HashSet<>()); } + // Identify segments generated by the appending job with a lower version than the current used segments for (final String version : versionToSegments.keySet()) { Set lowerVersionSegments = new HashSet<>(); for (final DataSegment segment : segments) { @@ -1177,6 +1190,7 @@ Map> allocateNewSegmentIds( } } + // Create a mapping from segment intervals to the set of lower version segments belonging to them Map> intervalToSegments = new HashMap<>(); for (DataSegment segment : lowerVersionSegments) { for (Interval interval : intervalToSegments.keySet()) { @@ -1205,9 +1219,11 @@ Map> allocateNewSegmentIds( } for (Interval interval : intervalToSegments.keySet()) { + // For each interval, fetch the pending segments Set pendingSegments = new HashSet<>( getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval) ); + // For each interval, fetch all the used segments Collection committedSegments = retrieveAllUsedSegmentsForIntervalsWithHandle( handle, dataSource, @@ -1230,6 +1246,7 @@ Map> allocateNewSegmentIds( version, NumberedPartialShardSpec.instance() ); + // allocate new segment id SegmentIdWithShardSpec newId = createNewSegment( request, dataSource, @@ -1238,6 +1255,7 @@ Map> allocateNewSegmentIds( committedMaxId, pendingSegments ); + // Add to set of pending segments so that shard specs are computed taking the new id into account pendingSegments.add(newId); retVal.get(segment).add(newId); } @@ -1698,6 +1716,19 @@ private Set announceHistoricalSegmentBatch( return toInsertSegments; } + + /** + * Commit all the segments corresponding to a replace job, + * AND also carry forward the segment metadata for segments associated with the given REPLACE locks. + * To be called within a single transaction + * + * @param handle - handle to execute queries + * @param segments - complete set of segments to be committed by the replacing job + * @param replaceLocks - The REPLACE locks held for the datasource over the specified set of segments + * @return The set of segments committed by the replace job as well as the new metadata for segments carried forward + * @throws IOException + */ + private Set commitReplaceSegmentBatch( final Handle handle, final Set segments, @@ -1705,6 +1736,7 @@ private Set commitReplaceSegmentBatch( ) throws IOException { final Set toInsertSegments = new HashSet<>(); + final Set retVal = new HashSet<>(); try { Set existedSegments = segmentExistsBatch(handle, segments); log.info("Found these segments already exist in DB: %s", existedSegments); @@ -1722,9 +1754,11 @@ private Set commitReplaceSegmentBatch( MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE ); + // Commit the set of segments generated by the replace job PreparedBatch preparedBatch = handle.prepareBatch(insertSegmentQuery); for (List partition : partitionedSegments) { for (DataSegment segment : partition) { + retVal.add(segment); final String now = DateTimes.nowUtc().toString(); preparedBatch.add() .bind("id", segment.getId().toString()) @@ -1755,11 +1789,13 @@ private Set commitReplaceSegmentBatch( } } + // Find the committed appended segment ids associated with one of the given REPLACE locks Map segmentsToBeForwarded = getAppendedSegmentIds( handle, segments.iterator().next().getDataSource(), replaceLocks ); + // Compute the core partition count as well as the max partition num Map intervalToCorePartition = new HashMap<>(); Map intervalToPartitionNum = new HashMap<>(); for (DataSegment segment : toInsertSegments) { @@ -1771,6 +1807,7 @@ private Set commitReplaceSegmentBatch( ); intervalToPartitionNum.put(segment.getInterval(), maxPartitionNum); } + final List>> forwardSegmentsBatch = Lists.partition( new ArrayList<>(segmentsToBeForwarded.entrySet()), MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE @@ -1780,10 +1817,14 @@ private Set commitReplaceSegmentBatch( for (Map.Entry entry : batch) { batchMap.put(entry.getKey(), entry.getValue()); } + // Find the segment metadata for the segment ids in this batch List oldSegments = retrieveSegmentsById(handle, batchMap.keySet()); for (DataSegment oldSegment : oldSegments) { Interval newInterval = oldSegment.getInterval(); for (DataSegment segment : toInsertSegments) { + // Check for interval overlap between each segment inserted with the segment in the batch. + // Overlapping segments can only be carried forward if their intervals are enclosed + // by those of the existing segments. if (segment.getInterval().overlaps(newInterval)) { if (segment.getInterval().contains(newInterval)) { newInterval = segment.getInterval(); @@ -1796,10 +1837,12 @@ private Set commitReplaceSegmentBatch( } } TaskLockInfo lock = batchMap.get(oldSegment.getId().toString()); + // Update states for generating unique shard specs final int partitionNum = intervalToPartitionNum.get(newInterval) + 1; final int numCorePartitions = intervalToCorePartition.get(newInterval); ShardSpec shardSpec = new NumberedShardSpec(partitionNum, numCorePartitions); intervalToPartitionNum.put(newInterval, partitionNum); + // Create the new segment metadata for the already appended segment using the same load spec DataSegment newSegment = new DataSegment( oldSegment.getDataSource(), newInterval, @@ -1811,6 +1854,7 @@ private Set commitReplaceSegmentBatch( oldSegment.getBinaryVersion(), oldSegment.getSize() ); + retVal.add(newSegment); final String now = DateTimes.nowUtc().toString(); preparedBatch.add() .bind("id", newSegment.getId().toString()) @@ -1839,9 +1883,19 @@ private Set commitReplaceSegmentBatch( throw e; } - return toInsertSegments; + return retVal; } + /** + * Commit segments created by the appending job as well as metadata copied to all greater versions with used segments. + * Also, used to commit metadata to associate segments with the locks they were held with. + * Should be called within a transaction + * @param handle - handle to interact with the metadata store + * @param segments - Segments allocated by the task as well as the metadata copies for higher versions + * @param appendSegmentLockMap - Mapping of segment to the replace lock held at the time of the commit, if any. + * @return The set of committed segments + * @throws IOException + */ private Set commitAppendSegmentBatch( final Handle handle, final Set segments, @@ -1866,6 +1920,7 @@ private Set commitAppendSegmentBatch( MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE ); + // Commit the set of segments created by the appending job in batches PreparedBatch preparedBatch = handle.prepareBatch(insertSegmentQuery); for (List partition : partitionedSegments) { for (DataSegment segment : partition) { @@ -1910,6 +1965,7 @@ private Set commitAppendSegmentBatch( if (appendSegmentLockMap == null) { appendSegmentLockMap = new HashMap<>(); } + // Process segment to replace lock mapping in batches final List>> appendSegmentPartitions = Lists.partition( new ArrayList<>(appendSegmentLockMap.entrySet()), MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE From 5981130a0fc684a18e515d5aeeef44cad7071c9c Mon Sep 17 00:00:00 2001 From: Amatya Date: Wed, 23 Aug 2023 07:56:20 +0530 Subject: [PATCH 16/43] Fix intellij inspections --- .../common/task/ConcurrentReplaceAndAppendTest.java | 2 +- .../druid/indexing/common/task/IngestionTestBase.java | 2 +- .../overlord/IndexerMetadataStorageCoordinator.java | 4 ++-- .../IndexerSQLMetadataStorageCoordinator.java | 11 +++++++---- .../IndexerSQLMetadataStorageCoordinatorTest.java | 2 +- 5 files changed, 12 insertions(+), 9 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java index 6ec1c78de2f2..bf69b0dcfb2d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java @@ -66,7 +66,7 @@ public class ConcurrentReplaceAndAppendTest extends IngestionTestBase private static final WorkerConfig WORKER_CONFIG = new WorkerConfig().setCapacity(10); @Before - public void setup() throws Exception + public void setup() { final TaskConfig taskConfig = new TaskConfigBuilder().build(); final TaskActionClientFactory taskActionClientFactory = createActionClientFactory(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 474ca004e805..45bdba00d7d7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -226,7 +226,7 @@ public TaskActionToolbox createTaskActionToolbox() ); } - public TaskToolbox createTaskToolbox(TaskConfig config, Task task) throws IOException + public TaskToolbox createTaskToolbox(TaskConfig config, Task task) { return new TaskToolbox.Builder() .config(config) diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 058e178a9b87..450bc6617e6e 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -282,12 +282,12 @@ SegmentPublishResult commitAppendSegments( @Nullable DataSourceMetadata startMetadata, @Nullable DataSourceMetadata endMetadata, @Nullable Map segmentLockMap - ) throws IOException; + ); SegmentPublishResult commitReplaceSegments( Set segments, @Nullable Set taskLockInfos - ) throws IOException; + ); /** * Retrieves data source's metadata from the metadata store. Returns null if there is no metadata. diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index b5784e5cf96a..ed52e56c008f 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -487,8 +487,9 @@ public SegmentPublishResult commitAppendSegments( 0, SQLMetadataConnector.DEFAULT_MAX_TRIES ); - for (DataSegment segment : segmentToNewMetadataMap.keySet()) { - for (SegmentIdWithShardSpec newId : segmentToNewMetadataMap.get(segment)) { + for (Map.Entry> entry : segmentToNewMetadataMap.entrySet()) { + final DataSegment segment = entry.getKey(); + for (SegmentIdWithShardSpec newId : entry.getValue()) { DataSegment newSegment = new DataSegment( newId.getDataSource(), newId.getInterval(), @@ -1193,9 +1194,11 @@ Map> allocateNewSegmentIds( // Create a mapping from segment intervals to the set of lower version segments belonging to them Map> intervalToSegments = new HashMap<>(); for (DataSegment segment : lowerVersionSegments) { - for (Interval interval : intervalToSegments.keySet()) { + for (final Map.Entry> entry : intervalToSegments.entrySet()) { + final Interval interval = entry.getKey(); + final Set segmentsForInterval = entry.getValue(); if (interval.contains(segment.getInterval())) { - intervalToSegments.get(interval).add(segment); + segmentsForInterval.add(segment); break; } } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 552be7966eba..027412edaa23 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -496,7 +496,7 @@ private Boolean insertIntoSegmentVersionsTable(Map se final DataSegment segment = entry.getKey(); final TaskLockInfo lock = entry.getValue(); preparedBatch.add() - .bind("id", segment.getId().toString() + ":" + lock.hashCode()) + .bind("id", segment.getId() + ":" + lock.hashCode()) .bind("dataSource", segment.getDataSource()) .bind("start", lock.getInterval().getStartMillis()) .bind("end", lock.getInterval().getEndMillis()) From 073bc26d92c06dffabbca5b5839954ec5bd3fab4 Mon Sep 17 00:00:00 2001 From: Amatya Date: Wed, 23 Aug 2023 08:08:40 +0530 Subject: [PATCH 17/43] Remove unnecessary exception handling --- .../common/task/ConcurrentReplaceAndAppendTest.java | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java index bf69b0dcfb2d..4ece665b071f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java @@ -52,7 +52,6 @@ import org.junit.Before; import org.junit.Test; -import java.io.IOException; import java.util.Collection; import java.util.TreeSet; import java.util.function.Function; @@ -1234,12 +1233,7 @@ public TaskToolbox build(Function decoratorFn, Task task @Override public TaskToolbox build(TaskConfig config, Task task) { - try { - return createTaskToolbox(config, task); - } - catch (IOException e) { - throw new RuntimeException(e); - } + return createTaskToolbox(config, task); } } } From b271f1ffc280f26981bb1622e18530b7d5e05c06 Mon Sep 17 00:00:00 2001 From: Amatya Date: Wed, 23 Aug 2023 08:53:23 +0530 Subject: [PATCH 18/43] Fix failing tests --- .../apache/druid/indexing/common/task/IngestionTestBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 45bdba00d7d7..d8d1e802b9a4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -120,7 +120,7 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest public void setUpIngestionTestBase() throws IOException { EmittingLogger.registerEmitter(new NoopServiceEmitter()); - baseDir = temporaryFolder.newFolder("baseWorkDir"); + temporaryFolder.create(); final SQLMetadataConnector connector = derbyConnectorRule.getConnector(); connector.createTaskTables(); From 725265cd8daaef0bbc01b793d6fd3b1abdc8023c Mon Sep 17 00:00:00 2001 From: Amatya Date: Wed, 23 Aug 2023 11:26:53 +0530 Subject: [PATCH 19/43] Add new test task class --- .../SegmentTransactionalReplaceAction.java | 21 +- .../druid/indexing/common/task/IndexTask.java | 2 +- .../parallel/ParallelIndexSupervisorTask.java | 2 +- .../common/task/CommandExecutingTask.java | 332 ++++++++++++++++++ .../task/ConcurrentReplaceAndAppendTest.java | 146 ++++++++ .../task/batch/parallel/ReplaceTask.java | 2 +- .../IndexerSQLMetadataStorageCoordinator.java | 5 +- 7 files changed, 486 insertions(+), 24 deletions(-) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandExecutingTask.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index 06e24db2f4ba..2a6d25548647 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -33,7 +33,6 @@ import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; -import javax.annotation.Nullable; import java.util.HashSet; import java.util.Map; import java.util.Set; @@ -45,41 +44,26 @@ */ public class SegmentTransactionalReplaceAction implements TaskAction { - /** - * Set of segments that was fully overshadowed by new segments, {@link SegmentTransactionalReplaceAction#segments} - */ - @Nullable - private final Set segmentsToBeOverwritten; /** * Set of segments to be inserted into metadata storage */ private final Set segments; public static SegmentTransactionalReplaceAction create( - @Nullable Set segmentsToBeOverwritten, Set segmentsToPublish ) { - return new SegmentTransactionalReplaceAction(segmentsToBeOverwritten, segmentsToPublish); + return new SegmentTransactionalReplaceAction(segmentsToPublish); } @JsonCreator private SegmentTransactionalReplaceAction( - @JsonProperty("segmentsToBeOverwritten") @Nullable Set segmentsToBeOverwritten, @JsonProperty("segments") Set segments ) { - this.segmentsToBeOverwritten = segmentsToBeOverwritten; this.segments = ImmutableSet.copyOf(segments); } - @JsonProperty - @Nullable - public Set getSegmentsToBeOverwritten() - { - return segmentsToBeOverwritten; - } - @JsonProperty public Set getSegments() { @@ -159,8 +143,7 @@ public boolean isAudited() public String toString() { return "SegmentTransactionalReplaceAction{" + - "segmentsToBeOverwritten=" + SegmentUtils.commaSeparatedIdentifiers(segmentsToBeOverwritten) + - ", segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + + "segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + '}'; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 1225748a635a..7ba1b968310d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -920,7 +920,7 @@ private TaskStatus generateAndPublishSegments( switch (lockType) { case REPLACE: return toolbox.getTaskActionClient().submit( - SegmentTransactionalReplaceAction.create(segmentsToBeOverwritten, segmentsToPublish) + SegmentTransactionalReplaceAction.create(segmentsToPublish) ); case APPEND: return toolbox.getTaskActionClient().submit( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index ee56fdcb9e1b..faee80f27ff0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -1178,7 +1178,7 @@ private void publishSegments( switch (lockType) { case REPLACE: return toolbox.getTaskActionClient().submit( - SegmentTransactionalReplaceAction.create(segmentsToBeOverwritten, segmentsToPublish) + SegmentTransactionalReplaceAction.create(segmentsToPublish) ); case APPEND: return toolbox.getTaskActionClient().submit( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandExecutingTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandExecutingTask.java new file mode 100644 index 000000000000..aa2468826c7e --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandExecutingTask.java @@ -0,0 +1,332 @@ +/* + * 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.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.indexer.TaskStatus; +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.TaskToolbox; +import org.apache.druid.indexing.common.actions.SegmentAllocateAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; +import org.apache.druid.indexing.common.actions.SurrogateTaskActionClient; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; +import org.apache.druid.indexing.common.config.TaskConfig; +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.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; +import org.apache.druid.server.security.ResourceAction; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.NumberedPartialShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nonnull; +import java.io.IOException; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; + +/** + * A test Task which can run a Runnable for its isReady and runTask methods + */ +public class CommandExecutingTask extends AbstractTask +{ + private final Interval interval; + private final Granularity segmentGranularity; + private final TaskLockType lockType; + private final int priority; + private final CountDownLatch readyLatch = new CountDownLatch(1); + private final CountDownLatch readyComplete = new CountDownLatch(1); + private final CountDownLatch runLatch = new CountDownLatch(1); + private final CountDownLatch runComplete = new CountDownLatch(1); + private final CountDownLatch publishLatch = new CountDownLatch(1); + private String version; + private Runnable command; + + private final Set pendingSegments = new HashSet<>(); + private final AtomicInteger sequenceId = new AtomicInteger(0); + + private TaskToolbox toolbox; + + public CommandExecutingTask( + String id, + String dataSource, + Interval interval, + Granularity segmentGranularity, + Map context, + IngestionMode ingestionMode + ) + { + super( + id == null ? StringUtils.format("command_%s_%s", DateTimes.nowUtc(), UUID.randomUUID().toString()) : id, + dataSource == null ? "none" : dataSource, + context, + ingestionMode + ); + this.interval = interval; + this.segmentGranularity = segmentGranularity; + this.lockType = TaskLockType.valueOf(getContextValue(Tasks.TASK_LOCK_TYPE, "EXCLUSIVE")); + this.priority = getContextValue(Tasks.PRIORITY_KEY, 0); + } + + @Override + public String getType() + { + return "command"; + } + + @Nonnull + @Override + public Set getInputSourceResources() + { + return ImmutableSet.of(); + } + + @Override + public String setup(TaskToolbox toolbox) throws Exception + { + readyLatch.await(); + while (!isReady(toolbox.getTaskActionClient())) { + Thread.sleep(100); + } + return null; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + return tryTimeChunkLockSingleInterval( + new SurrogateTaskActionClient(getId(), taskActionClient), + interval, + lockType + ); + } + + private boolean tryTimeChunkLockSingleInterval(TaskActionClient client, Interval interval, TaskLockType lockType) + throws IOException + { + final TaskLock lock = client.submit(new TimeChunkLockTryAcquireAction(lockType, interval)); + if (lock == null) { + return false; + } + if (lock.isRevoked()) { + throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", interval)); + } + version = lock.getVersion(); + return true; + } + + @Override + public void stopGracefully(TaskConfig taskConfig) + { + } + + @Override + public TaskStatus runTask(TaskToolbox toolbox) throws Exception + { + this.toolbox = toolbox; + + readyComplete.countDown(); + + runLatch.await(); + + try { + command.run(); + return TaskStatus.success(getId()); + } + catch (Exception e) { + return TaskStatus.failure(getId(), "Failed to execute the command."); + } + } + + @Override + public void cleanUp(TaskToolbox toolbox, TaskStatus taskStatus) throws Exception + { + super.cleanUp(toolbox, taskStatus); + runComplete.countDown(); + } + + @Override + public int getPriority() + { + return priority; + } + + public void awaitRunComplete() throws InterruptedException + { + runComplete.await(); + } + + public void awaitReadyComplete() throws InterruptedException + { + readyComplete.await(); + } + + public void markReady() + { + readyLatch.countDown(); + } + + public void beginPublish() + { + publishLatch.countDown(); + } + + public void runCommand(Runnable command) + { + this.command = command; + runLatch.countDown(); + } + + public Set appendSegments(Set newSegments) throws Exception + { + publishLatch.await(); + if (!TaskLockType.APPEND.equals(lockType)) { + throw new ISE("appendSegments can be called only with an APPEND lock."); + } + final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> + toolbox.getTaskActionClient().submit( + SegmentTransactionalAppendAction.create(segmentsToPublish, null, null) + ); + return publisher.publishSegments( + Collections.emptySet(), + newSegments, + Function.identity(), + null + ).getSegments(); + } + + public Set convertPendingSegments(Set pendingSegments) + { + final Set segments = new HashSet<>(); + for (SegmentIdWithShardSpec pendingSegment : pendingSegments) { + final SegmentId id = pendingSegment.asSegmentId(); + segments.add( + new DataSegment( + id, + ImmutableMap.of(id.toString(), id.toString()), + ImmutableList.of(), + ImmutableList.of(), + pendingSegment.getShardSpec(), + null, + 0, + 0 + ) + ); + } + return segments; + } + + + public Set replaceSegments(Set newSegments) throws Exception + { + if (!TaskLockType.REPLACE.equals(lockType)) { + throw new ISE("replaceSegments can be called only with a REPLACEk lock."); + } + final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> + toolbox.getTaskActionClient().submit( + SegmentTransactionalReplaceAction.create(segmentsToPublish) + ); + return publisher.publishSegments( + null, + newSegments, + Function.identity(), + null + ).getSegments(); + } + + public Set createCorePartitions(int numSegmentsPerInterval) + { + final Set newSegments = new HashSet<>(); + for (int i = 0; i < numSegmentsPerInterval; i++) { + for (Interval subInterval : segmentGranularity.getIterable(interval)) { + final ShardSpec shardSpec = new NumberedShardSpec(i, numSegmentsPerInterval); + final SegmentId segmentId = SegmentId.of(getDataSource(), subInterval, version, shardSpec); + newSegments.add( + new DataSegment( + segmentId, + ImmutableMap.of(segmentId.toString(), segmentId.toString()), + ImmutableList.of(), + ImmutableList.of(), + shardSpec, + null, + 0, + 0 + ) + ); + } + } + return newSegments; + } + + public SegmentIdWithShardSpec allocateOrGetSegmentForTimestamp(String timestamp) + { + final DateTime time = DateTimes.of(timestamp); + for (SegmentIdWithShardSpec pendingSegment : pendingSegments) { + if (pendingSegment.getInterval().contains(time)) { + return pendingSegment; + } + } + return allocateNewSegmentForDate(time); + } + + private SegmentIdWithShardSpec allocateNewSegmentForDate(DateTime time) + { + try { + SegmentAllocateAction allocateAction = new SegmentAllocateAction( + getDataSource(), + time, + Granularities.NONE, + segmentGranularity, + getId() + "_" + sequenceId.getAndIncrement(), + null, + false, + NumberedPartialShardSpec.instance(), + LockGranularity.TIME_CHUNK, + lockType + ); + final SegmentIdWithShardSpec id = toolbox.getTaskActionClient().submit(allocateAction); + pendingSegments.add(id); + return id; + } + catch (Exception e) { + return null; + } + + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java index 4ece665b071f..d88586b963bc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.common.task; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import org.apache.druid.indexing.common.MultipleFileTaskReportFileWriter; import org.apache.druid.indexing.common.TaskStorageDirTracker; import org.apache.druid.indexing.common.TaskToolbox; @@ -43,6 +44,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.DruidNode; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.tasklogs.NoopTaskLogs; @@ -53,12 +55,18 @@ import org.junit.Test; import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; import java.util.TreeSet; import java.util.function.Function; import java.util.stream.Collectors; public class ConcurrentReplaceAndAppendTest extends IngestionTestBase { + private static final Map REPLACE_CONTEXT = ImmutableMap.of(Tasks.TASK_LOCK_TYPE, "REPLACE"); + private static final Map APPEND_CONTEXT = ImmutableMap.of(Tasks.TASK_LOCK_TYPE, "APPEND"); + private TaskQueue taskQueue; private TaskActionClient taskActionClient; @@ -95,6 +103,144 @@ public void setup() taskQueue.start(); } + @Test + public void testCommandExecutingTask() throws Exception + { + + CommandExecutingTask replaceTask0 = new CommandExecutingTask( + "replace0", + "DS", + Intervals.of("2023/2024"), + Granularities.YEAR, + REPLACE_CONTEXT, + AbstractTask.IngestionMode.REPLACE + ); + Runnable runReplaceTask0 = new Runnable() + { + @Override + public void run() + { + try { + final Set segments = replaceTask0.createCorePartitions(1); + replaceTask0.replaceSegments(segments); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + }; + + CommandExecutingTask appendTask0 = new CommandExecutingTask( + "append0", + "DS", + Intervals.of("2023/2024"), + Granularities.YEAR, + APPEND_CONTEXT, + AbstractTask.IngestionMode.APPEND + ); + Runnable runAppendTask0 = new Runnable() + { + @Override + public void run() + { + try { + final Set pendingSegments = new HashSet<>(); + pendingSegments.add( + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-01") + ); + final Set segments = appendTask0.convertPendingSegments(pendingSegments); + appendTask0.appendSegments(segments); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + }; + + CommandExecutingTask replaceTask1 = new CommandExecutingTask( + "replace1", + "DS", + Intervals.of("2023/2024"), + Granularities.YEAR, + REPLACE_CONTEXT, + AbstractTask.IngestionMode.REPLACE + ); + Runnable runReplaceTask1 = new Runnable() + { + @Override + public void run() + { + try { + final Set segments = replaceTask1.createCorePartitions(1); + replaceTask1.replaceSegments(segments); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + }; + + CommandExecutingTask replaceTask2 = new CommandExecutingTask( + "replace2", + "DS", + Intervals.of("2023/2024"), + Granularities.YEAR, + REPLACE_CONTEXT, + AbstractTask.IngestionMode.REPLACE + ); + Runnable runReplaceTask2 = new Runnable() + { + @Override + public void run() + { + try { + final Set segments = replaceTask2.createCorePartitions(1); + replaceTask2.replaceSegments(segments); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + }; + + + // Create a set of initial segments + taskQueue.add(replaceTask0); + replaceTask0.markReady(); + replaceTask0.runCommand(runReplaceTask0); + replaceTask0.awaitRunComplete(); + verifySegmentCount(1, 1); + verifyTaskSuccess(replaceTask0); + + // Append task begins and allocates pending segments + taskQueue.add(appendTask0); + appendTask0.markReady(); + appendTask0.awaitReadyComplete(); + appendTask0.runCommand(runAppendTask0); + + // New replace task starts and ends before the appending task finishes + taskQueue.add(replaceTask1); + replaceTask1.markReady(); + replaceTask1.runCommand(runReplaceTask1); + replaceTask1.awaitRunComplete(); + verifySegmentCount(2, 1); + verifyTaskSuccess(replaceTask1); + + taskQueue.add(replaceTask2); + replaceTask2.markReady(); + replaceTask2.awaitReadyComplete(); + + appendTask0.beginPublish(); + appendTask0.awaitRunComplete(); + verifySegmentCount(4, 2); + verifyTaskSuccess(appendTask0); + + replaceTask2.runCommand(runReplaceTask2); + replaceTask2.awaitRunComplete(); + verifySegmentCount(6, 2); + verifyTaskSuccess(replaceTask2); + } + @Test public void test() throws Exception { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java index 83f7befd5dbd..485a1dc6cec6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java @@ -212,7 +212,7 @@ private boolean publishSegments(TaskToolbox toolbox, Set oldSegment { final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> toolbox.getTaskActionClient().submit( - SegmentTransactionalReplaceAction.create(segmentsToBeOverwritten, segmentsToPublish) + SegmentTransactionalReplaceAction.create(segmentsToPublish) ); return publisher.publishSegments( oldSegments, diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index ed52e56c008f..a8dd15d905a9 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1221,7 +1221,8 @@ Map> allocateNewSegmentIds( } } - for (Interval interval : intervalToSegments.keySet()) { + for (Map.Entry> entry : intervalToSegments.entrySet()) { + final Interval interval = entry.getKey(); // For each interval, fetch the pending segments Set pendingSegments = new HashSet<>( getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval) @@ -1242,7 +1243,7 @@ Map> allocateNewSegmentIds( committedMaxId = SegmentIdWithShardSpec.fromDataSegment(committedSegment); } } - for (DataSegment segment : intervalToSegments.get(interval)) { + for (DataSegment segment : entry.getValue()) { SegmentCreateRequest request = new SegmentCreateRequest( segment.getId() + version, null, From 41c3cbe35f80e3d2f2195d88bba74f9c348c7d8b Mon Sep 17 00:00:00 2001 From: Amatya Date: Wed, 23 Aug 2023 13:37:47 +0530 Subject: [PATCH 20/43] A replacing task must look only at its own replace locks during commit --- .../SegmentTransactionalReplaceAction.java | 16 ++++++++++------ .../druid/indexing/common/actions/TaskLocks.java | 3 ++- 2 files changed, 12 insertions(+), 7 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index 2a6d25548647..c7434b6672f6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableSet; +import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; @@ -33,8 +34,6 @@ import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; -import java.util.HashSet; -import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -84,9 +83,14 @@ public TypeReference getReturnTypeReference() @Override public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) { - String datasource = task.getDataSource(); - final Map segmentToReplaceLock - = TaskLocks.findReplaceLocksCoveringSegments(datasource, toolbox.getTaskLockbox(), segments); + // Find the active replace locks held only by this task + final Set replaceLocksForTask = + toolbox.getTaskLockbox() + .findLocksForTask(task) + .stream() + .filter(taskLock -> !taskLock.isRevoked() && TaskLockType.REPLACE.equals(taskLock.getType())) + .map(TaskLocks::toLockInfo) + .collect(Collectors.toSet()); final SegmentPublishResult retVal; try { @@ -97,7 +101,7 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) .onValidLocks( () -> toolbox.getIndexerMetadataStorageCoordinator().commitReplaceSegments( segments, - new HashSet<>(segmentToReplaceLock.values()) + replaceLocksForTask ) ) .onInvalidLocks( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java index 3b4fa067bc96..28bfad2b39b5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java @@ -139,7 +139,8 @@ public static Map findReplaceLocksCoveringSegments( ).add(segment) ); - final Set replaceLocks = taskLockbox.getAllReplaceLocksForDatasource(datasource).stream() + final Set replaceLocks = taskLockbox.getAllReplaceLocksForDatasource(datasource) + .stream() .map(TaskLocks::toLockInfo) .collect(Collectors.toSet()); From 5963be3a0e4edacb23822a8a3995e92873a2a8ac Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 25 Aug 2023 12:41:33 +0530 Subject: [PATCH 21/43] Remove some unused fields --- .../SegmentTransactionalAppendAction.java | 40 +- .../SegmentTransactionalReplaceAction.java | 12 +- .../indexing/common/actions/TaskLocks.java | 13 + .../druid/indexing/common/task/IndexTask.java | 2 +- .../parallel/ParallelIndexSupervisorTask.java | 2 +- .../common/task/CommandExecutingTask.java | 2 +- .../task/ConcurrentReplaceAndAppendTest.java | 85 ++- .../task/batch/parallel/AppendTask.java | 2 +- ...TestIndexerMetadataStorageCoordinator.java | 5 +- .../IndexerMetadataStorageCoordinator.java | 6 +- .../IndexerSQLMetadataStorageCoordinator.java | 495 ++++++++---------- ...exerSQLMetadataStorageCoordinatorTest.java | 6 +- 12 files changed, 284 insertions(+), 386 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java index d9399dab438d..39425a55d641 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java @@ -22,18 +22,15 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; -import com.google.common.collect.ImmutableSet; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; -import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.TaskLockInfo; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; -import javax.annotation.Nullable; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -46,30 +43,19 @@ public class SegmentTransactionalAppendAction implements TaskAction segments; - @Nullable - private final DataSourceMetadata startMetadata; - @Nullable - private final DataSourceMetadata endMetadata; - public static SegmentTransactionalAppendAction create( - Set segments, - @Nullable DataSourceMetadata startMetadata, - @Nullable DataSourceMetadata endMetadata + Set segments ) { - return new SegmentTransactionalAppendAction(segments, startMetadata, endMetadata); + return new SegmentTransactionalAppendAction(segments); } @JsonCreator private SegmentTransactionalAppendAction( - @JsonProperty("segments") @Nullable Set segments, - @JsonProperty("startMetadata") @Nullable DataSourceMetadata startMetadata, - @JsonProperty("endMetadata") @Nullable DataSourceMetadata endMetadata + @JsonProperty("segments") Set segments ) { - this.segments = segments == null ? ImmutableSet.of() : ImmutableSet.copyOf(segments); - this.startMetadata = startMetadata; - this.endMetadata = endMetadata; + this.segments = segments; } @JsonProperty @@ -78,20 +64,6 @@ public Set getSegments() return segments; } - @JsonProperty - @Nullable - public DataSourceMetadata getStartMetadata() - { - return startMetadata; - } - - @JsonProperty - @Nullable - public DataSourceMetadata getEndMetadata() - { - return endMetadata; - } - @Override public TypeReference getReturnTypeReference() { @@ -119,8 +91,6 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) .onValidLocks( () -> toolbox.getIndexerMetadataStorageCoordinator().commitAppendSegments( segments, - startMetadata, - endMetadata, segmentToReplaceLock ) ) @@ -165,8 +135,6 @@ public String toString() { return "SegmentTransactionalAppendAction{" + "segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + - ", startMetadata=" + startMetadata + - ", endMetadata=" + endMetadata + '}'; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index c7434b6672f6..ff165fdbb24e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableSet; -import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; @@ -83,14 +82,11 @@ public TypeReference getReturnTypeReference() @Override public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) { + TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); + // Find the active replace locks held only by this task - final Set replaceLocksForTask = - toolbox.getTaskLockbox() - .findLocksForTask(task) - .stream() - .filter(taskLock -> !taskLock.isRevoked() && TaskLockType.REPLACE.equals(taskLock.getType())) - .map(TaskLocks::toLockInfo) - .collect(Collectors.toSet()); + final Set replaceLocksForTask + = TaskLocks.findReplaceLocksHeldByTask(task, toolbox.getTaskLockbox()); final SegmentPublishResult retVal; try { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java index 28bfad2b39b5..6fa2ae7dfe40 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java @@ -159,6 +159,19 @@ public static Map findReplaceLocksCoveringSegments( return segmentToReplaceLock; } + /** + * Finds the active locks of type {@link TaskLockType#REPLACE} held by this task. + */ + public static Set findReplaceLocksHeldByTask(Task task, TaskLockbox taskLockbox) + { + return taskLockbox + .findLocksForTask(task) + .stream() + .filter(taskLock -> !taskLock.isRevoked() && TaskLockType.REPLACE.equals(taskLock.getType())) + .map(TaskLocks::toLockInfo) + .collect(Collectors.toSet()); + } + public static TaskLockInfo toLockInfo(TaskLock taskLock) { return new TaskLockInfo(taskLock.getInterval(), taskLock.getVersion()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 7ba1b968310d..beabb9490a7d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -924,7 +924,7 @@ private TaskStatus generateAndPublishSegments( ); case APPEND: return toolbox.getTaskActionClient().submit( - SegmentTransactionalAppendAction.create(segmentsToPublish, null, null) + SegmentTransactionalAppendAction.create(segmentsToPublish) ); default: return toolbox.getTaskActionClient().submit( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index faee80f27ff0..179f31614061 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -1182,7 +1182,7 @@ private void publishSegments( ); case APPEND: return toolbox.getTaskActionClient().submit( - SegmentTransactionalAppendAction.create(segmentsToPublish, null, null) + SegmentTransactionalAppendAction.create(segmentsToPublish) ); default: return toolbox.getTaskActionClient().submit( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandExecutingTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandExecutingTask.java index aa2468826c7e..07386bd7d75c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandExecutingTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandExecutingTask.java @@ -221,7 +221,7 @@ public Set appendSegments(Set newSegments) throws Exce } final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> toolbox.getTaskActionClient().submit( - SegmentTransactionalAppendAction.create(segmentsToPublish, null, null) + SegmentTransactionalAppendAction.create(segmentsToPublish) ); return publisher.publishSegments( Collections.emptySet(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java index d88586b963bc..f810c3b81754 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java @@ -106,7 +106,6 @@ public void setup() @Test public void testCommandExecutingTask() throws Exception { - CommandExecutingTask replaceTask0 = new CommandExecutingTask( "replace0", "DS", @@ -115,18 +114,13 @@ public void testCommandExecutingTask() throws Exception REPLACE_CONTEXT, AbstractTask.IngestionMode.REPLACE ); - Runnable runReplaceTask0 = new Runnable() - { - @Override - public void run() - { - try { - final Set segments = replaceTask0.createCorePartitions(1); - replaceTask0.replaceSegments(segments); - } - catch (Exception e) { - throw new RuntimeException(e); - } + Runnable runReplaceTask0 = () -> { + try { + final Set segments = replaceTask0.createCorePartitions(1); + replaceTask0.replaceSegments(segments); + } + catch (Exception e) { + throw new RuntimeException(e); } }; @@ -138,22 +132,17 @@ public void run() APPEND_CONTEXT, AbstractTask.IngestionMode.APPEND ); - Runnable runAppendTask0 = new Runnable() - { - @Override - public void run() - { - try { - final Set pendingSegments = new HashSet<>(); - pendingSegments.add( - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-01") - ); - final Set segments = appendTask0.convertPendingSegments(pendingSegments); - appendTask0.appendSegments(segments); - } - catch (Exception e) { - throw new RuntimeException(e); - } + Runnable runAppendTask0 = () -> { + try { + final Set pendingSegments = new HashSet<>(); + pendingSegments.add( + appendTask0.allocateOrGetSegmentForTimestamp("2023-01-01") + ); + final Set segments = appendTask0.convertPendingSegments(pendingSegments); + appendTask0.appendSegments(segments); + } + catch (Exception e) { + throw new RuntimeException(e); } }; @@ -165,18 +154,13 @@ public void run() REPLACE_CONTEXT, AbstractTask.IngestionMode.REPLACE ); - Runnable runReplaceTask1 = new Runnable() - { - @Override - public void run() - { - try { - final Set segments = replaceTask1.createCorePartitions(1); - replaceTask1.replaceSegments(segments); - } - catch (Exception e) { - throw new RuntimeException(e); - } + Runnable runReplaceTask1 = () -> { + try { + final Set segments = replaceTask1.createCorePartitions(1); + replaceTask1.replaceSegments(segments); + } + catch (Exception e) { + throw new RuntimeException(e); } }; @@ -188,18 +172,13 @@ public void run() REPLACE_CONTEXT, AbstractTask.IngestionMode.REPLACE ); - Runnable runReplaceTask2 = new Runnable() - { - @Override - public void run() - { - try { - final Set segments = replaceTask2.createCorePartitions(1); - replaceTask2.replaceSegments(segments); - } - catch (Exception e) { - throw new RuntimeException(e); - } + Runnable runReplaceTask2 = () -> { + try { + final Set segments = replaceTask2.createCorePartitions(1); + replaceTask2.replaceSegments(segments); + } + catch (Exception e) { + throw new RuntimeException(e); } }; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java index 9a728e70f26e..db67ebab4b0b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java @@ -231,7 +231,7 @@ private boolean publishSegments(TaskToolbox toolbox, Set newSegment { final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> toolbox.getTaskActionClient().submit( - SegmentTransactionalAppendAction.create(segmentsToPublish, null, null) + SegmentTransactionalAppendAction.create(segmentsToPublish) ); return publisher.publishSegments( Collections.emptySet(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index 3040f6a26623..4e42a9c98019 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -170,12 +170,9 @@ public SegmentPublishResult commitReplaceSegments( @Override public SegmentPublishResult commitAppendSegments( Set segments, - DataSourceMetadata oldCommitMetadata, - DataSourceMetadata newCommitMetadata, - @Nullable Map segmentLockMap + Map segmentToReplaceLock ) { - // Don't actually compare metadata, just do it! return SegmentPublishResult.ok(commitSegments(segments)); } diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 450bc6617e6e..a65cbe76b12f 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -279,14 +279,12 @@ SegmentPublishResult commitSegmentsAndMetadata( SegmentPublishResult commitAppendSegments( Set segments, - @Nullable DataSourceMetadata startMetadata, - @Nullable DataSourceMetadata endMetadata, - @Nullable Map segmentLockMap + Map segmentToReplaceLock ); SegmentPublishResult commitReplaceSegments( Set segments, - @Nullable Set taskLockInfos + Set taskLockInfos ); /** diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index a8dd15d905a9..4efa84398f7f 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -110,8 +110,6 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor private final MetadataStorageTablesConfig dbTables; private final SQLMetadataConnector connector; - private final String insertSegmentQuery; - @Inject public IndexerSQLMetadataStorageCoordinator( ObjectMapper jsonMapper, @@ -122,12 +120,6 @@ public IndexerSQLMetadataStorageCoordinator( this.jsonMapper = jsonMapper; this.dbTables = dbTables; this.connector = connector; - this.insertSegmentQuery = 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)", - dbTables.getSegmentsTable(), - connector.getQuoteString() - ); } @LifecycleStart @@ -258,14 +250,14 @@ private Set getPendingSegmentsForIntervalWithHandle( final ResultIterator dbSegments = handle.createQuery( - StringUtils.format( - // This query might fail if the year has a different number of digits - // See https://github.com/apache/druid/pull/11582 for a similar issue - // Using long for these timestamps instead of varchar would give correct time comparisons - "SELECT payload FROM %1$s WHERE dataSource = :dataSource AND start < :end and %2$send%2$s > :start", - dbTables.getPendingSegmentsTable(), connector.getQuoteString() - ) - ) + StringUtils.format( + // This query might fail if the year has a different number of digits + // See https://github.com/apache/druid/pull/11582 for a similar issue + // Using long for these timestamps instead of varchar would give correct time comparisons + "SELECT payload FROM %1$s WHERE dataSource = :dataSource AND start < :end and %2$send%2$s > :start", + dbTables.getPendingSegmentsTable(), connector.getQuoteString() + ) + ) .bind("dataSource", dataSource) .bind("start", interval.getStart().toString()) .bind("end", interval.getEnd().toString()) @@ -416,22 +408,12 @@ public SegmentPublishResult inTransaction( @Override public SegmentPublishResult commitReplaceSegments( final Set segments, - @Nullable Set taskLockInfos + final Set taskLockInfos ) { - if (segments.isEmpty()) { - throw new IllegalArgumentException("segment set must not be empty"); - } - - final String dataSource = segments.iterator().next().getDataSource(); - for (DataSegment segment : segments) { - if (!dataSource.equals(segment.getDataSource())) { - throw new IllegalArgumentException("segments must all be from the same dataSource"); - } - } + verifySegmentsToCommit(segments); final AtomicBoolean definitelyNotUpdated = new AtomicBoolean(false); - try { return connector.retryTransaction( (handle, transactionStatus) -> { @@ -439,7 +421,7 @@ public SegmentPublishResult commitReplaceSegments( definitelyNotUpdated.set(false); final Set inserted = commitReplaceSegmentBatch(handle, segments, taskLockInfos); - return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); + return SegmentPublishResult.ok(inserted); }, 3, getSqlMetadataMaxRetry() @@ -458,97 +440,45 @@ public SegmentPublishResult commitReplaceSegments( @Override public SegmentPublishResult commitAppendSegments( final Set segments, - @Nullable final DataSourceMetadata startMetadata, - @Nullable DataSourceMetadata endMetadata, - @Nullable Map segmentLockMap + final Map segmentToReplaceLock ) { - if (segments.isEmpty()) { - throw new IllegalArgumentException("No segments to append"); - } + verifySegmentsToCommit(segments); + // For each append segment, check if extra versions need to be inserted into the DB + // This may be required if a REPLACE job has run after the allocation of these append segments final String dataSource = segments.iterator().next().getDataSource(); - for (DataSegment segment : segments) { - if (!dataSource.equals(segment.getDataSource())) { - throw new IllegalArgumentException("All segments to append must belong to the same dataSource"); - } - } - - if ((startMetadata == null && endMetadata != null) || (startMetadata != null && endMetadata == null)) { - throw new IllegalArgumentException("Start and end metadata must either be both null or both non-null"); - } - - // Find which segments are used (i.e. not overshadowed). - Set newSegments = new HashSet<>(segments); - // Allocate new segment ids for the same segments based on the presence of used segments with higher versions. - // It happens when replace jobs have completed with greater versions between the allocation and commit of segments. - final Map> segmentToNewMetadataMap = connector.retryTransaction( - (handle, transactionStatus) -> allocateNewSegmentIds(handle, dataSource, segments), + final Map> segmentToNewIds = connector.retryTransaction( + (handle, transactionStatus) -> allocateExtraIdsForAppendSegments(handle, dataSource, segments), 0, SQLMetadataConnector.DEFAULT_MAX_TRIES ); - for (Map.Entry> entry : segmentToNewMetadataMap.entrySet()) { + + // Insert entries for all required versions of the append segments into the DB + final Set allSegmentsToInsert = new HashSet<>(segments); + for (Map.Entry> entry : segmentToNewIds.entrySet()) { final DataSegment segment = entry.getKey(); for (SegmentIdWithShardSpec newId : entry.getValue()) { - DataSegment newSegment = new DataSegment( - newId.getDataSource(), - newId.getInterval(), - newId.getVersion(), - segment.getLoadSpec(), - segment.getDimensions(), - segment.getMetrics(), - newId.getShardSpec(), - segment.getBinaryVersion(), - segment.getSize() - ); - newSegments.add(newSegment); + DataSegment newSegment = DataSegment.builder(segment) + .version(newId.getVersion()) + .shardSpec(newId.getShardSpec()) + .build(); + allSegmentsToInsert.add(newSegment); } } - final AtomicBoolean definitelyNotUpdated = new AtomicBoolean(false); try { return connector.retryTransaction( (handle, transactionStatus) -> { - // Set definitelyNotUpdated back to false upon retrying. - definitelyNotUpdated.set(false); - - if (startMetadata != null) { - final DataStoreMetadataUpdateResult result = updateDataSourceMetadataWithHandle( - handle, - dataSource, - startMetadata, - endMetadata - ); - - if (result.isFailed()) { - // Metadata was definitely not updated. - transactionStatus.setRollbackOnly(); - definitelyNotUpdated.set(true); - - if (result.canRetry()) { - throw new RetryTransactionException(result.getErrorMsg()); - } else { - throw new RuntimeException(result.getErrorMsg()); - } - } - } - - // Commit not only the segments generated by the appending job initially, - // but also those ids which need to be appended to future versions. - final Set inserted = commitAppendSegmentBatch(handle, newSegments, segmentLockMap); - return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); + insertSegmentLockVersions(segmentToReplaceLock, handle); + return SegmentPublishResult.ok(insertSegments(allSegmentsToInsert, handle)); }, 3, getSqlMetadataMaxRetry() ); } catch (CallbackFailedException e) { - if (definitelyNotUpdated.get()) { - return SegmentPublishResult.fail(e.getMessage()); - } else { - // Must throw exception if we are not sure if we updated or not. - throw e; - } + return SegmentPublishResult.fail(e.getMessage()); } } @@ -1131,74 +1061,67 @@ private void insertPendingSegmentIntoMetastore( } /** - * Find versions to which appending segments must be committed to and create appropriate segment identifiers for them - * @param handle - Handle to interact with the metadata store - * @param dataSource - datasource to which the appending job is writing - * @param segments - Set of segments generated by the append job - * @return Mapping of input segments to all the Ids with which its metadata must be committed - * @throws IOException + * Allocates and returns any extra versions that need to be committed for the + * given append segments. + *

+ * + * This is typically needed when a REPLACE task started and finished after + * these append segments had already been allocated. As such, + * there would be some used segments in the DB with versions higher than these + * append segments. */ @VisibleForTesting - Map> allocateNewSegmentIds( + Map> allocateExtraIdsForAppendSegments( Handle handle, String dataSource, - Set segments + Set segmentsToAppend ) throws IOException { - if (segments.isEmpty()) { + if (segmentsToAppend.isEmpty()) { return Collections.emptyMap(); } - // Store the mapping from version to segment / interval for which used segments already exist - Map> versionToSegments = new HashMap<>(); - Map> versionToIntervals = new HashMap<>(); - Collection segmentIntervals = segments.stream() - .map(DataSegment::getInterval) - .collect(Collectors.toSet()); + final Set appendIntervals = segmentsToAppend.stream() + .map(DataSegment::getInterval) + .collect(Collectors.toSet()); + + // Find all used segments whose interval overlaps with any of append intervals + final Map> committedVersionToSegments = new HashMap<>(); + final Map> committedVersionToIntervals = new HashMap<>(); try (final CloseableIterator iterator = SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper) - .retrieveUsedSegments( - dataSource, - segmentIntervals - ) + .retrieveUsedSegments(dataSource, appendIntervals) ) { while (iterator.hasNext()) { final DataSegment segment = iterator.next(); - versionToSegments.computeIfAbsent(segment.getVersion(), v -> new HashSet<>()) - .add(segment); - versionToIntervals.computeIfAbsent(segment.getVersion(), v -> new HashSet<>()) - .add(segment.getInterval()); + committedVersionToSegments.computeIfAbsent(segment.getVersion(), v -> new HashSet<>()) + .add(segment); + committedVersionToIntervals.computeIfAbsent(segment.getVersion(), v -> new HashSet<>()) + .add(segment.getInterval()); } } - // Maps segment to its new metadata with higher versions - Map> retVal = new HashMap<>(); - for (DataSegment segment : segments) { - retVal.put(segment, new HashSet<>()); - } + final Map> segmentToNewIds = new HashMap<>(); - // Identify segments generated by the appending job with a lower version than the current used segments - for (final String version : versionToSegments.keySet()) { - Set lowerVersionSegments = new HashSet<>(); - for (final DataSegment segment : segments) { + // Identify the append segments that need to be carried forward to each new version + for (final String version : committedVersionToSegments.keySet()) { + final Set segmentsToCarryForward = new HashSet<>(); + for (final DataSegment segment : segmentsToAppend) { if (segment.getVersion().compareTo(version) < 0) { - for (final Interval interval : versionToIntervals.get(version)) { + for (final Interval interval : committedVersionToIntervals.get(version)) { if (interval.overlaps(segment.getInterval())) { - lowerVersionSegments.add(segment); + segmentsToCarryForward.add(segment); break; } } } } - // Create a mapping from segment intervals to the set of lower version segments belonging to them - Map> intervalToSegments = new HashMap<>(); - for (DataSegment segment : lowerVersionSegments) { - for (final Map.Entry> entry : intervalToSegments.entrySet()) { - final Interval interval = entry.getKey(); - final Set segmentsForInterval = entry.getValue(); - if (interval.contains(segment.getInterval())) { - segmentsForInterval.add(segment); + final Map> intervalToCarryForwardSegments = new HashMap<>(); + for (DataSegment segment : segmentsToCarryForward) { + for (final Map.Entry> entry : intervalToCarryForwardSegments.entrySet()) { + if (entry.getKey().contains(segment.getInterval())) { + entry.getValue().add(segment); break; } } @@ -1209,7 +1132,7 @@ Map> allocateNewSegmentIds( ); for (DataSegment overlappingSegment : overlappingSegments) { if (overlappingSegment.getInterval().contains(segment.getInterval())) { - intervalToSegments.computeIfAbsent(overlappingSegment.getInterval(), itvl -> new HashSet<>()) + intervalToCarryForwardSegments.computeIfAbsent(overlappingSegment.getInterval(), itvl -> new HashSet<>()) .add(segment); } else { throw new ISE( @@ -1221,7 +1144,7 @@ Map> allocateNewSegmentIds( } } - for (Map.Entry> entry : intervalToSegments.entrySet()) { + for (Map.Entry> entry : intervalToCarryForwardSegments.entrySet()) { final Interval interval = entry.getKey(); // For each interval, fetch the pending segments Set pendingSegments = new HashSet<>( @@ -1261,11 +1184,12 @@ Map> allocateNewSegmentIds( ); // Add to set of pending segments so that shard specs are computed taking the new id into account pendingSegments.add(newId); - retVal.get(segment).add(newId); + segmentToNewIds.computeIfAbsent(segment, s -> new HashSet<>()) + .add(newId); } } } - return retVal; + return segmentToNewIds; } private Map createNewSegments( @@ -1680,21 +1604,21 @@ private Set announceHistoricalSegmentBatch( MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE ); - PreparedBatch preparedBatch = handle.prepareBatch(insertSegmentQuery); + PreparedBatch preparedBatch = handle.prepareBatch(buildSqlToInsertSegments()); for (List partition : partitionedSegments) { for (DataSegment segment : partition) { final String now = DateTimes.nowUtc().toString(); preparedBatch.add() - .bind("id", segment.getId().toString()) - .bind("dataSource", segment.getDataSource()) - .bind("created_date", now) - .bind("start", segment.getInterval().getStart().toString()) - .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) - .bind("version", segment.getVersion()) - .bind("used", usedSegments.contains(segment)) - .bind("payload", jsonMapper.writeValueAsBytes(segment)) - .bind("used_status_last_updated", now); + .bind("id", segment.getId().toString()) + .bind("dataSource", segment.getDataSource()) + .bind("created_date", now) + .bind("start", segment.getInterval().getStart().toString()) + .bind("end", segment.getInterval().getEnd().toString()) + .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) + .bind("version", segment.getVersion()) + .bind("used", usedSegments.contains(segment)) + .bind("payload", jsonMapper.writeValueAsBytes(segment)) + .bind("used_status_last_updated", now); } final int[] affectedRows = preparedBatch.execute(); final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); @@ -1702,9 +1626,9 @@ private Set announceHistoricalSegmentBatch( log.infoSegments(partition, "Published segments to DB"); } else { final List failedToPublish = IntStream.range(0, partition.size()) - .filter(i -> affectedRows[i] != 1) - .mapToObj(partition::get) - .collect(Collectors.toList()); + .filter(i -> affectedRows[i] != 1) + .mapToObj(partition::get) + .collect(Collectors.toList()); throw new ISE( "Failed to publish segments to DB: %s", SegmentUtils.commaSeparatedIdentifiers(failedToPublish) @@ -1726,26 +1650,22 @@ private Set announceHistoricalSegmentBatch( * AND also carry forward the segment metadata for segments associated with the given REPLACE locks. * To be called within a single transaction * - * @param handle - handle to execute queries - * @param segments - complete set of segments to be committed by the replacing job * @param replaceLocks - The REPLACE locks held for the datasource over the specified set of segments * @return The set of segments committed by the replace job as well as the new metadata for segments carried forward - * @throws IOException */ - private Set commitReplaceSegmentBatch( final Handle handle, final Set segments, - @Nullable Set replaceLocks + final Set replaceLocks ) throws IOException { final Set toInsertSegments = new HashSet<>(); final Set retVal = new HashSet<>(); try { - Set existedSegments = segmentExistsBatch(handle, segments); - log.info("Found these segments already exist in DB: %s", existedSegments); + final Set existingSegments = segmentExistsBatch(handle, segments); + log.debug("There are [%d] segment IDs already present in the metadata store.", existingSegments.size()); for (DataSegment segment : segments) { - if (!existedSegments.contains(segment.getId().toString())) { + if (!existingSegments.contains(segment.getId().toString())) { toInsertSegments.add(segment); } } @@ -1759,7 +1679,7 @@ private Set commitReplaceSegmentBatch( ); // Commit the set of segments generated by the replace job - PreparedBatch preparedBatch = handle.prepareBatch(insertSegmentQuery); + PreparedBatch preparedBatch = handle.prepareBatch(buildSqlToInsertSegments()); for (List partition : partitionedSegments) { for (DataSegment segment : partition) { retVal.add(segment); @@ -1891,122 +1811,140 @@ private Set commitReplaceSegmentBatch( } /** - * Commit segments created by the appending job as well as metadata copied to all greater versions with used segments. - * Also, used to commit metadata to associate segments with the locks they were held with. - * Should be called within a transaction - * @param handle - handle to interact with the metadata store - * @param segments - Segments allocated by the task as well as the metadata copies for higher versions - * @param appendSegmentLockMap - Mapping of segment to the replace lock held at the time of the commit, if any. - * @return The set of committed segments - * @throws IOException + * Verifies that: + *

    + *
  • The set of segments being committed is non-empty.
  • + *
  • All segments belong to the same datasource.
  • + *
*/ - private Set commitAppendSegmentBatch( - final Handle handle, - final Set segments, - @Nullable Map appendSegmentLockMap - ) throws IOException + private void verifySegmentsToCommit(Collection segments) { - final Set toInsertSegments = new HashSet<>(); - try { - Set existedSegments = segmentExistsBatch(handle, segments); - log.info("Found these segments already exist in DB: %s", existedSegments); - for (DataSegment segment : segments) { - if (!existedSegments.contains(segment.getId().toString())) { - toInsertSegments.add(segment); - } + if (segments.isEmpty()) { + throw new IllegalArgumentException("No segment to commit"); + } + + final String dataSource = segments.iterator().next().getDataSource(); + for (DataSegment segment : segments) { + if (!dataSource.equals(segment.getDataSource())) { + throw new IllegalArgumentException("Segments to commit must all belong to the same datasource"); } + } + } - // SELECT -> INSERT can fail due to races; callers must be prepared to retry. - // Avoiding ON DUPLICATE KEY since it's not portable. - // Avoiding try/catch since it may cause inadvertent transaction-splitting. - final List> partitionedSegments = Lists.partition( - new ArrayList<>(toInsertSegments), - MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE - ); + /** + * Inserts the given segments into the DB in batches of size + * {@link #MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE} and returns the set of + * segments actually inserted. + *

+ * This method avoids inserting segment IDs which already exist in the DB. + * Callers of this method might need to retry as INSERT followed by SELECT + * might fail due to race conditions. + */ + private Set insertSegments(Set segments, Handle handle) + throws IOException + { + // Do not insert segment IDs which already exist + Set existingSegmentIds = segmentExistsBatch(handle, segments); + final Set segmentsToInsert = segments.stream().filter( + s -> !existingSegmentIds.contains(s.getId().toString()) + ).collect(Collectors.toSet()); + + // Insert the segments in batches of manageable size + final List> partitionedSegments = Lists.partition( + new ArrayList<>(segmentsToInsert), + MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE + ); - // Commit the set of segments created by the appending job in batches - PreparedBatch preparedBatch = handle.prepareBatch(insertSegmentQuery); - for (List partition : partitionedSegments) { - for (DataSegment segment : partition) { - final String now = DateTimes.nowUtc().toString(); - preparedBatch.add() - .bind("id", segment.getId().toString()) - .bind("dataSource", segment.getDataSource()) - .bind("created_date", now) - .bind("start", segment.getInterval().getStart().toString()) - .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) - .bind("version", segment.getVersion()) - .bind("used", true) - .bind("payload", jsonMapper.writeValueAsBytes(segment)) - .bind("used_status_last_updated", now); - } - final int[] affectedInsertRows = preparedBatch.execute(); + final PreparedBatch batch = handle.prepareBatch(buildSqlToInsertSegments()); + for (List partition : partitionedSegments) { + for (DataSegment segment : partition) { + final String now = DateTimes.nowUtc().toString(); + batch.add() + .bind("id", segment.getId().toString()) + .bind("dataSource", segment.getDataSource()) + .bind("created_date", now) + .bind("start", segment.getInterval().getStart().toString()) + .bind("end", segment.getInterval().getEnd().toString()) + .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) + .bind("version", segment.getVersion()) + .bind("used", true) + .bind("payload", jsonMapper.writeValueAsBytes(segment)) + .bind("used_status_last_updated", now); + } - final boolean succeeded = Arrays.stream(affectedInsertRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); - if (succeeded) { - log.infoSegments(partition, "Published segments to DB"); - } else { - final List failedToPublish = IntStream.range(0, partition.size()) - .filter(i -> affectedInsertRows[i] != 1) - .mapToObj(partition::get) - .collect(Collectors.toList()); - throw new ISE( - "Failed to publish segments to DB: %s", - SegmentUtils.commaSeparatedIdentifiers(failedToPublish) - ); + final int[] affectedRows = batch.execute(); + + final List failedInserts = new ArrayList<>(); + for (int i = 0; i < partition.size(); ++i) { + if (affectedRows[i] != 1) { + failedInserts.add(partition.get(i)); } } + if (failedInserts.isEmpty()) { + log.infoSegments(partition, "Published segments to DB"); + } else { + throw new ISE( + "Failed to publish segments to DB: %s", + SegmentUtils.commaSeparatedIdentifiers(failedInserts) + ); + } + } - PreparedBatch appendBatch = handle.prepareBatch( - StringUtils.format( - "INSERT INTO %1$s (id, dataSource, start, %2$send%2$s, segment_id, lock_version) " - + "VALUES (:id, :dataSource, :start, :end, :segment_id, :lock_version)", - dbTables.getSegmentVersionsTable(), - connector.getQuoteString() - ) - ); - if (appendSegmentLockMap == null) { - appendSegmentLockMap = new HashMap<>(); + return segmentsToInsert; + } + + /** + * Inserts entries into the segment lock version table in batches of size + * {@link #MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE}. + */ + private void insertSegmentLockVersions( + Map segmentToReplaceLock, + Handle handle + ) + { + if (segmentToReplaceLock.isEmpty()) { + return; + } + + final PreparedBatch batch = handle.prepareBatch( + StringUtils.format( + "INSERT INTO %1$s (id, dataSource, start, %2$send%2$s, segment_id, lock_version) " + + "VALUES (:id, :dataSource, :start, :end, :segment_id, :lock_version)", + dbTables.getSegmentVersionsTable(), connector.getQuoteString() + ) + ); + + final List>> partitions = Lists.partition( + new ArrayList<>(segmentToReplaceLock.entrySet()), + MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE + ); + for (List> partition : partitions) { + for (Map.Entry entry : partition) { + DataSegment segment = entry.getKey(); + TaskLockInfo lock = entry.getValue(); + batch.add() + .bind("id", segment.getId() + ":" + lock.hashCode()) + .bind("dataSource", segment.getDataSource()) + .bind("start", lock.getInterval().getStartMillis()) + .bind("end", lock.getInterval().getEndMillis()) + .bind("segment_id", segment.getId().toString()) + .bind("lock_version", lock.getVersion()); } - // Process segment to replace lock mapping in batches - final List>> appendSegmentPartitions = Lists.partition( - new ArrayList<>(appendSegmentLockMap.entrySet()), - MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE - ); - for (List> partition : appendSegmentPartitions) { - for (Map.Entry entry : partition) { - DataSegment segment = entry.getKey(); - TaskLockInfo lock = entry.getValue(); - appendBatch.add() - .bind("id", segment.getId() + ":" + lock.hashCode()) - .bind("dataSource", segment.getDataSource()) - .bind("start", lock.getInterval().getStartMillis()) - .bind("end", lock.getInterval().getEndMillis()) - .bind("segment_id", segment.getId().toString()) - .bind("lock_version", lock.getVersion()); - } - final int[] affectedAppendRows = appendBatch.execute(); - final boolean succeeded = Arrays.stream(affectedAppendRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); - if (!succeeded) { - final List failedToForward = IntStream.range(0, partition.size()) - .filter(i -> affectedAppendRows[i] != 1) - .mapToObj(partition::get) - .map(x -> x.getKey()) - .collect(Collectors.toList()); - throw new ISE( - "Failed to forward appended segments to DB: %s", - SegmentUtils.commaSeparatedIdentifiers(failedToForward) - ); + final int[] affectedAppendRows = batch.execute(); + + final List failedInserts = new ArrayList<>(); + for (int i = 0; i < partition.size(); ++i) { + if (affectedAppendRows[i] != 1) { + failedInserts.add(partition.get(i).getKey()); } } + if (failedInserts.size() > 0) { + throw new ISE( + "Failed to insert lock versions in DB: %s", + SegmentUtils.commaSeparatedIdentifiers(failedInserts) + ); + } } - catch (Exception e) { - log.errorSegments(segments, "Exception inserting segment metadata"); - throw e; - } - - return toInsertSegments; } private List retrieveSegmentsById(Handle handle, Set segmentIds) @@ -2027,6 +1965,18 @@ private List retrieveSegmentsById(Handle handle, Set segmen return Lists.newArrayList(resultIterator); } + private String buildSqlToInsertSegments() + { + return 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)", + dbTables.getSegmentsTable(), + connector.getQuoteString() + ); + } + @VisibleForTesting Map getAppendedSegmentIds( Handle handle, @@ -2041,8 +1991,7 @@ Map getAppendedSegmentIds( sb.append( StringUtils.format( "SELECT segment_id, start, %1$send%1$s, lock_version FROM %2$s where dataSource = :dataSource AND (", - connector.getQuoteString(), - dbTables.getSegmentVersionsTable() + connector.getQuoteString(), dbTables.getSegmentVersionsTable() ) ); diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 027412edaa23..0cd11326b265 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -635,9 +635,7 @@ public void testAllocateNewSegmentIds() segmentsToBeProcessed.addAll(month1); segmentsToBeProcessed.addAll(month2); final Map> segmentToNewIds = derbyConnector.retryWithHandle( - handle -> { - return coordinator.allocateNewSegmentIds(handle, "foo", segmentsToBeProcessed); - } + handle -> coordinator.allocateExtraIdsForAppendSegments(handle, "foo", segmentsToBeProcessed) ); for (DataSegment segment : day1) { @@ -726,7 +724,7 @@ public void testCommitAppendSegments() allSegments.add(segment); } - coordinator.commitAppendSegments(allSegments, null, null, segmentLockMap); + coordinator.commitAppendSegments(allSegments, segmentLockMap); Assert.assertEquals( allSegments.stream().map(DataSegment::getId).map(SegmentId::toString).collect(Collectors.toSet()), From 71023ca0a39a417198beaf9c61612989191016a7 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 29 Aug 2023 18:14:19 +0530 Subject: [PATCH 22/43] Add some javadocs --- .../SegmentTransactionalAppendAction.java | 2 + .../SegmentTransactionalReplaceAction.java | 6 +- .../indexing/common/actions/TaskLocks.java | 6 +- ...TestIndexerMetadataStorageCoordinator.java | 12 +- .../MetadataStorageTablesConfigTest.java | 1 + .../test/resources/test.runtime.properties | 1 + .../IndexerMetadataStorageCoordinator.java | 23 +- .../druid/indexing/overlord/TaskLockInfo.java | 16 +- .../IndexerSQLMetadataStorageCoordinator.java | 570 +++++++----------- .../druid/metadata/SQLMetadataConnector.java | 30 +- ...exerSQLMetadataStorageCoordinatorTest.java | 22 +- 11 files changed, 296 insertions(+), 393 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java index 39425a55d641..52496eac7782 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java @@ -78,6 +78,8 @@ public TypeReference getReturnTypeReference() @Override public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) { + TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); + final String datasource = task.getDataSource(); final Map segmentToReplaceLock = TaskLocks.findReplaceLocksCoveringSegments(datasource, toolbox.getTaskLockbox(), segments); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index ff165fdbb24e..232951d7eee6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -95,10 +95,8 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), CriticalAction.builder() .onValidLocks( - () -> toolbox.getIndexerMetadataStorageCoordinator().commitReplaceSegments( - segments, - replaceLocksForTask - ) + () -> toolbox.getIndexerMetadataStorageCoordinator() + .commitReplaceSegments(segments, replaceLocksForTask) ) .onInvalidLocks( () -> SegmentPublishResult.fail( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java index 6fa2ae7dfe40..439345c3d5f9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java @@ -146,11 +146,11 @@ public static Map findReplaceLocksCoveringSegments( final Map segmentToReplaceLock = new HashMap<>(); - intervalToSegments.forEach((interval, segmentsForInterval) -> { + intervalToSegments.forEach((interval, segmentsInInterval) -> { // For each interval, find the lock that covers it, if any for (TaskLockInfo lock : replaceLocks) { if (lock.getInterval().contains(interval)) { - segmentsForInterval.forEach(s -> segmentToReplaceLock.put(s, lock)); + segmentsInInterval.forEach(s -> segmentToReplaceLock.put(s, lock)); return; } } @@ -174,7 +174,7 @@ public static Set findReplaceLocksHeldByTask(Task task, TaskLockbo public static TaskLockInfo toLockInfo(TaskLock taskLock) { - return new TaskLockInfo(taskLock.getInterval(), taskLock.getVersion()); + return new TaskLockInfo(taskLock.getGroupId(), taskLock.getInterval(), taskLock.getVersion()); } public static List findLocksForSegments( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index 222fe7191b18..42995c84bf9c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -159,21 +159,21 @@ public Map allocatePendingSegments @Override public SegmentPublishResult commitReplaceSegments( - Set segments, - Set taskLockInfos + Set replaceSegments, + Set locksHeldByReplaceTask ) { // Don't actually compare metadata, just do it! - return SegmentPublishResult.ok(commitSegments(segments)); + return SegmentPublishResult.ok(commitSegments(replaceSegments)); } @Override public SegmentPublishResult commitAppendSegments( - Set segments, - Map segmentToReplaceLock + Set appendSegments, + Map appendSegmentToReplaceLock ) { - return SegmentPublishResult.ok(commitSegments(segments)); + return SegmentPublishResult.ok(commitSegments(appendSegments)); } @Override diff --git a/processing/src/test/java/org/apache/druid/guice/MetadataStorageTablesConfigTest.java b/processing/src/test/java/org/apache/druid/guice/MetadataStorageTablesConfigTest.java index 40f2d4805b97..c48570bb0c7a 100644 --- a/processing/src/test/java/org/apache/druid/guice/MetadataStorageTablesConfigTest.java +++ b/processing/src/test/java/org/apache/druid/guice/MetadataStorageTablesConfigTest.java @@ -80,5 +80,6 @@ public ObjectMapper jsonMapper() ); Assert.assertEquals(props.getProperty("druid.metadata.storage.tables.dataSource"), config.getDataSourceTable()); Assert.assertEquals(props.getProperty("druid.metadata.storage.tables.supervisors"), config.getSupervisorTable()); + Assert.assertEquals(props.getProperty("druid.metadata.storage.tables.segmentVersions"), config.getSegmentVersionsTable()); } } diff --git a/processing/src/test/resources/test.runtime.properties b/processing/src/test/resources/test.runtime.properties index 554e711636aa..80ed65f863c5 100644 --- a/processing/src/test/resources/test.runtime.properties +++ b/processing/src/test/resources/test.runtime.properties @@ -27,5 +27,6 @@ druid.metadata.storage.tables.taskLock=fff_tasklock druid.metadata.storage.tables.audit=ggg_audit druid.metadata.storage.tables.dataSource=hhh_dataSource druid.metadata.storage.tables.supervisors=iii_supervisors +druid.metadata.storage.tables.segmentVersions=jjj_segmentVersions druid.query.segmentMetadata.defaultAnalysisTypes=["cardinality", "size"] druid.query.segmentMetadata.defaultHistory=P2W diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 89f4c51d1772..cf0a095a2c7a 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -277,14 +277,29 @@ SegmentPublishResult commitSegmentsAndMetadata( @Nullable DataSourceMetadata endMetadata ) throws IOException; + /** + * Commits segments created by an APPEND task. If any REPLACE segment that + * overlaps with these {@code appendSegments} was committed while this append + * task was in progress, the {@code appendSegments} are also added to the + * version of the replace segment. + * + * @param appendSegments + * @param appendSegmentToReplaceLock + */ SegmentPublishResult commitAppendSegments( - Set segments, - Map segmentToReplaceLock + Set appendSegments, + Map appendSegmentToReplaceLock ); + /** + * Commits segments created by a REPLACE task. If any APPEND segment that + * overlaps with these {@code replaceSegments} was committed while this replace + * task was in progress, the append segments are also added to the version of + * these {@code replaceSegments}. + */ SegmentPublishResult commitReplaceSegments( - Set segments, - Set taskLockInfos + Set replaceSegments, + Set locksHeldByReplaceTask ); /** diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/TaskLockInfo.java b/server/src/main/java/org/apache/druid/indexing/overlord/TaskLockInfo.java index a6ea0fad0066..a7dcb0c68b96 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/TaskLockInfo.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/TaskLockInfo.java @@ -25,15 +25,22 @@ public class TaskLockInfo { + private final String groupId; private final Interval interval; private final String version; - public TaskLockInfo(Interval interval, String version) + public TaskLockInfo(String groupId, Interval interval, String version) { + this.groupId = groupId; this.interval = interval; this.version = version; } + public String getGroupId() + { + return groupId; + } + public Interval getInterval() { return interval; @@ -54,13 +61,14 @@ public boolean equals(Object o) return false; } TaskLockInfo that = (TaskLockInfo) o; - return Objects.equals(interval, that.getInterval()) && - Objects.equals(version, that.version); + return Objects.equals(groupId, that.groupId) + && Objects.equals(interval, that.interval) + && Objects.equals(version, that.version); } @Override public int hashCode() { - return Objects.hash(interval, version); + return Objects.hash(groupId, interval, version); } } diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index e35dac4ee3f9..6a6229c26b6c 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -94,6 +94,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.TreeMap; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -407,59 +408,55 @@ public SegmentPublishResult inTransaction( @Override public SegmentPublishResult commitReplaceSegments( - final Set segments, - final Set taskLockInfos + final Set replaceSegments, + final Set locksHeldByReplaceTask ) { - verifySegmentsToCommit(segments); + verifySegmentsToCommit(replaceSegments); - final AtomicBoolean definitelyNotUpdated = new AtomicBoolean(false); try { return connector.retryTransaction( (handle, transactionStatus) -> { - // Set definitelyNotUpdated back to false upon retrying. - definitelyNotUpdated.set(false); - - final Set inserted = commitReplaceSegmentBatch(handle, segments, taskLockInfos); - return SegmentPublishResult.ok(inserted); + final Set segmentsToInsert = new HashSet<>(replaceSegments); + segmentsToInsert.addAll( + getSegmentsToUpgradeOnReplace(handle, replaceSegments, locksHeldByReplaceTask) + ); + return SegmentPublishResult.ok( + insertSegments(handle, segmentsToInsert) + ); }, 3, getSqlMetadataMaxRetry() ); } catch (CallbackFailedException e) { - if (definitelyNotUpdated.get()) { - return SegmentPublishResult.fail(e.getMessage()); - } else { - // Must throw exception if we are not sure if we updated or not. - throw e; - } + return SegmentPublishResult.fail(e.getMessage()); } } @Override public SegmentPublishResult commitAppendSegments( - final Set segments, - final Map segmentToReplaceLock + final Set appendSegments, + final Map appendSegmentToReplaceLock ) { - verifySegmentsToCommit(segments); + verifySegmentsToCommit(appendSegments); - // For each append segment, check if extra versions need to be inserted into the DB - // This may be required if a REPLACE job has run after the allocation of these append segments - final String dataSource = segments.iterator().next().getDataSource(); + final String dataSource = appendSegments.iterator().next().getDataSource(); final Map> segmentToNewIds = connector.retryTransaction( - (handle, transactionStatus) -> allocateExtraIdsForAppendSegments(handle, dataSource, segments), + (handle, transactionStatus) + -> getSegmentsToUpgradeOnAppend(handle, dataSource, appendSegments), 0, SQLMetadataConnector.DEFAULT_MAX_TRIES ); - // Insert entries for all required versions of the append segments into the DB - final Set allSegmentsToInsert = new HashSet<>(segments); + // Create entries for all required versions of the append segments + final Set allSegmentsToInsert = new HashSet<>(appendSegments); for (Map.Entry> entry : segmentToNewIds.entrySet()) { final DataSegment segment = entry.getKey(); for (SegmentIdWithShardSpec newId : entry.getValue()) { DataSegment newSegment = DataSegment.builder(segment) + .interval(newId.getInterval()) .version(newId.getVersion()) .shardSpec(newId.getShardSpec()) .build(); @@ -470,8 +467,8 @@ public SegmentPublishResult commitAppendSegments( try { return connector.retryTransaction( (handle, transactionStatus) -> { - insertSegmentLockVersions(segmentToReplaceLock, handle); - return SegmentPublishResult.ok(insertSegments(allSegmentsToInsert, handle)); + insertSegmentLockVersions(handle, appendSegmentToReplaceLock); + return SegmentPublishResult.ok(insertSegments(handle, allSegmentsToInsert)); }, 3, getSqlMetadataMaxRetry() @@ -1064,14 +1061,13 @@ private void insertPendingSegmentIntoMetastore( * Allocates and returns any extra versions that need to be committed for the * given append segments. *

- * * This is typically needed when a REPLACE task started and finished after * these append segments had already been allocated. As such, * there would be some used segments in the DB with versions higher than these * append segments. */ @VisibleForTesting - Map> allocateExtraIdsForAppendSegments( + Map> getSegmentsToUpgradeOnAppend( Handle handle, String dataSource, Set segmentsToAppend @@ -1081,115 +1077,133 @@ Map> allocateExtraIdsForAppendSegments( return Collections.emptyMap(); } - final Set appendIntervals = segmentsToAppend.stream() - .map(DataSegment::getInterval) - .collect(Collectors.toSet()); - - // Find all used segments whose interval overlaps with any of append intervals - final Map> committedVersionToSegments = new HashMap<>(); - final Map> committedVersionToIntervals = new HashMap<>(); - try (final CloseableIterator iterator = - SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper) - .retrieveUsedSegments(dataSource, appendIntervals) - ) { - while (iterator.hasNext()) { - final DataSegment segment = iterator.next(); - committedVersionToSegments.computeIfAbsent(segment.getVersion(), v -> new HashSet<>()) - .add(segment); - committedVersionToIntervals.computeIfAbsent(segment.getVersion(), v -> new HashSet<>()) - .add(segment.getInterval()); - } + final Set appendIntervals = new HashSet<>(); + final TreeMap> appendVersionToSegments = new TreeMap<>(); + for (DataSegment segment : segmentsToAppend) { + appendIntervals.add(segment.getInterval()); + appendVersionToSegments.computeIfAbsent(segment.getVersion(), v -> new HashSet<>()) + .add(segment); } - final Map> segmentToNewIds = new HashMap<>(); + // Fetch all used non-overshadowed segments that overlap with any of the append intervals + final Collection overlappingSegments + = retrieveUsedSegmentsForIntervals(dataSource, new ArrayList<>(appendIntervals), Segments.ONLY_VISIBLE); - // Identify the append segments that need to be carried forward to each new version - for (final String version : committedVersionToSegments.keySet()) { - final Set segmentsToCarryForward = new HashSet<>(); - for (final DataSegment segment : segmentsToAppend) { - if (segment.getVersion().compareTo(version) < 0) { - for (final Interval interval : committedVersionToIntervals.get(version)) { - if (interval.overlaps(segment.getInterval())) { - segmentsToCarryForward.add(segment); - break; - } - } - } - } + final Set committedVersions = new HashSet<>(); + final Map> committedIntervalToSegments = new HashMap<>(); + for (DataSegment segment : overlappingSegments) { + committedVersions.add(segment.getVersion()); + committedIntervalToSegments.computeIfAbsent(segment.getInterval(), i -> new HashSet<>()) + .add(segment); + } - final Map> intervalToCarryForwardSegments = new HashMap<>(); - for (DataSegment segment : segmentsToCarryForward) { - for (final Map.Entry> entry : intervalToCarryForwardSegments.entrySet()) { - if (entry.getKey().contains(segment.getInterval())) { - entry.getValue().add(segment); - break; - } - } - Collection overlappingSegments = retrieveUsedSegmentsForIntervals( - segment.getDataSource(), - ImmutableList.of(segment.getInterval()), - Segments.ONLY_VISIBLE + final Map> appendSegmentToNewIds = new HashMap<>(); + for (String version : committedVersions) { + Map> committedIntervalToCarrySegments + = getIntervalToCarrySegments(version, committedIntervalToSegments.keySet(), appendVersionToSegments); + for (Map.Entry> entry : committedIntervalToCarrySegments.entrySet()) { + computeNewAppendIdsForVersion( + handle, + version, + dataSource, + entry.getKey(), + entry.getValue(), + committedIntervalToSegments, + appendSegmentToNewIds ); - for (DataSegment overlappingSegment : overlappingSegments) { - if (overlappingSegment.getInterval().contains(segment.getInterval())) { - intervalToCarryForwardSegments.computeIfAbsent(overlappingSegment.getInterval(), itvl -> new HashSet<>()) - .add(segment); - } else { - throw new ISE( - "Existing segment interval[%s] conflicts with that of the new segment[%s]", - overlappingSegment.getInterval(), - segment.getInterval() - ); - } - } } + } - for (Map.Entry> entry : intervalToCarryForwardSegments.entrySet()) { - final Interval interval = entry.getKey(); - // For each interval, fetch the pending segments - Set pendingSegments = new HashSet<>( - getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval) - ); - // For each interval, fetch all the used segments - Collection committedSegments = retrieveAllUsedSegmentsForIntervalsWithHandle( - handle, - dataSource, - ImmutableList.of(interval) - ) - .stream() - .filter(s -> s.getVersion().equals(version) && s.getInterval().equals(interval)) - .collect(Collectors.toSet()); - SegmentIdWithShardSpec committedMaxId = null; - for (DataSegment committedSegment : committedSegments) { - if (committedMaxId == null - || committedMaxId.getShardSpec().getPartitionNum() < committedSegment.getShardSpec().getPartitionNum()) { - committedMaxId = SegmentIdWithShardSpec.fromDataSegment(committedSegment); - } - } - for (DataSegment segment : entry.getValue()) { - SegmentCreateRequest request = new SegmentCreateRequest( - segment.getId() + version, - null, - version, - NumberedPartialShardSpec.instance() - ); - // allocate new segment id - SegmentIdWithShardSpec newId = createNewSegment( - request, - dataSource, - interval, - version, - committedMaxId, - pendingSegments + return appendSegmentToNewIds; + } + + private Map> getIntervalToCarrySegments( + String version, + Set committedIntervals, + TreeMap> appendVersionToSegments + ) + { + // Find all the append segments with version strictly less than this version + final Set segmentsToCarryForward = new HashSet<>(); + + Map> candidateVersionToSegments = appendVersionToSegments.headMap(version); + for (Set segments : candidateVersionToSegments.values()) { + segmentsToCarryForward.addAll(segments); + } + + final Map> committedIntervalToAppendSegments = new HashMap<>(); + for (DataSegment appendSegment : segmentsToCarryForward) { + final Interval appendSegmentInterval = appendSegment.getInterval(); + for (Interval committedInterval : committedIntervals) { + if (committedInterval.contains(appendSegmentInterval)) { + committedIntervalToAppendSegments.computeIfAbsent(committedInterval, itvl -> new HashSet<>()) + .add(appendSegment); + break; + } else if (committedInterval.overlaps(appendSegmentInterval)) { + // Committed interval overlaps only partially + throw new ISE( + "Committed interval[%s] conflicts with interval[%s] of append segment[%s].", + committedInterval, appendSegmentInterval, appendSegment.getId() ); - // Add to set of pending segments so that shard specs are computed taking the new id into account - pendingSegments.add(newId); - segmentToNewIds.computeIfAbsent(segment, s -> new HashSet<>()) - .add(newId); } } } - return segmentToNewIds; + + return committedIntervalToAppendSegments; + } + + private void computeNewAppendIdsForVersion( + Handle handle, + String committedVersion, + String dataSource, + Interval interval, + Set carrySegments, + Map> committedIntervalToSegments, + Map> appendSegmentToNewIds + ) throws IOException + { + final Set committedSegments + = committedIntervalToSegments.getOrDefault(interval, Collections.emptySet()) + .stream() + .filter(s -> s.getVersion().equals(committedVersion)) + .collect(Collectors.toSet()); + + SegmentIdWithShardSpec committedMaxId = null; + for (DataSegment committedSegment : committedSegments) { + if (committedMaxId == null + || committedMaxId.getShardSpec().getPartitionNum() < committedSegment.getShardSpec().getPartitionNum()) { + committedMaxId = SegmentIdWithShardSpec.fromDataSegment(committedSegment); + } + } + + // Get pending segments for the new version, if any + final Set pendingSegments + = getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval); + + // Determine new IDs for each append segment by taking into account both + // committed and pending segments for this version + for (DataSegment segment : carrySegments) { + SegmentCreateRequest request = new SegmentCreateRequest( + segment.getId() + committedVersion, + null, + committedVersion, + NumberedPartialShardSpec.instance() + ); + // allocate new segment id + final SegmentIdWithShardSpec newId = createNewSegment( + request, + dataSource, + interval, + committedVersion, + committedMaxId, + pendingSegments + ); + + // Add to set of pending segments so that shard specs are computed taking the new id into account + pendingSegments.add(newId); + appendSegmentToNewIds.computeIfAbsent(segment, s -> new HashSet<>()) + .add(newId); + } } private Map createNewSegments( @@ -1644,170 +1658,65 @@ private Set announceHistoricalSegmentBatch( return toInsertSegments; } - - /** - * Commit all the segments corresponding to a replace job, - * AND also carry forward the segment metadata for segments associated with the given REPLACE locks. - * To be called within a single transaction - * - * @param replaceLocks - The REPLACE locks held for the datasource over the specified set of segments - * @return The set of segments committed by the replace job as well as the new metadata for segments carried forward - */ - private Set commitReplaceSegmentBatch( + private Set getSegmentsToUpgradeOnReplace( final Handle handle, - final Set segments, - final Set replaceLocks - ) throws IOException + final Set replaceSegments, + final Set locksHeldByReplaceTask + ) { - final Set toInsertSegments = new HashSet<>(); - final Set retVal = new HashSet<>(); - try { - final Set existingSegments = segmentExistsBatch(handle, segments); - log.debug("There are [%d] segment IDs already present in the metadata store.", existingSegments.size()); - for (DataSegment segment : segments) { - if (!existingSegments.contains(segment.getId().toString())) { - toInsertSegments.add(segment); - } - } - - // SELECT -> INSERT can fail due to races; callers must be prepared to retry. - // Avoiding ON DUPLICATE KEY since it's not portable. - // Avoiding try/catch since it may cause inadvertent transaction-splitting. - final List> partitionedSegments = Lists.partition( - new ArrayList<>(toInsertSegments), - MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE + // For each replace interval, find the number of core partitions and total partitions + final Map intervalToNumCorePartitions = new HashMap<>(); + final Map intervalToCurrentPartitionNum = new HashMap<>(); + for (DataSegment segment : replaceSegments) { + intervalToNumCorePartitions.put(segment.getInterval(), segment.getShardSpec().getNumCorePartitions()); + + int partitionNum = segment.getShardSpec().getPartitionNum(); + intervalToCurrentPartitionNum.compute( + segment.getInterval(), + (i, value) -> value == null ? partitionNum : Math.max(value, partitionNum) ); + } - // Commit the set of segments generated by the replace job - PreparedBatch preparedBatch = handle.prepareBatch(buildSqlToInsertSegments()); - for (List partition : partitionedSegments) { - for (DataSegment segment : partition) { - retVal.add(segment); - final String now = DateTimes.nowUtc().toString(); - preparedBatch.add() - .bind("id", segment.getId().toString()) - .bind("dataSource", segment.getDataSource()) - .bind("created_date", now) - .bind("start", segment.getInterval().getStart().toString()) - .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) - .bind("version", segment.getVersion()) - .bind("used", true) - .bind("payload", jsonMapper.writeValueAsBytes(segment)) - .bind("used_status_last_updated", now); - } - final int[] affectedInsertRows = preparedBatch.execute(); + final Map carryForwardSegmentToLockVersion + = getAppendSegmentsCommittedDuringTask(handle, locksHeldByReplaceTask); - final boolean succeeded = Arrays.stream(affectedInsertRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); - if (succeeded) { - log.infoSegments(partition, "Published segments to DB"); - } else { - final List failedToPublish = IntStream.range(0, partition.size()) - .filter(i -> affectedInsertRows[i] != 1) - .mapToObj(partition::get) - .collect(Collectors.toList()); + final List carryForwardSegments + = retrieveSegmentsById(handle, carryForwardSegmentToLockVersion.keySet()); + + final Set segmentsToInsert = new HashSet<>(); + for (DataSegment oldSegment : carryForwardSegments) { + Interval newInterval = oldSegment.getInterval(); + for (DataSegment segment : replaceSegments) { + final Interval segmentInterval = segment.getInterval(); + if (segmentInterval.contains(newInterval)) { + newInterval = segmentInterval; + break; + } else if (segmentInterval.overlaps(newInterval)) { throw new ISE( - "Failed to publish segments to DB: %s", - SegmentUtils.commaSeparatedIdentifiers(failedToPublish) + "Incompatible segment intervals for commit: [%s] and [%s].", + newInterval, segmentInterval ); } } - // Find the committed appended segment ids associated with one of the given REPLACE locks - Map segmentsToBeForwarded = getAppendedSegmentIds( - handle, - segments.iterator().next().getDataSource(), - replaceLocks + // Compute shard spec for the new version of the segment + final int partitionNum = intervalToCurrentPartitionNum.compute( + newInterval, + (i, value) -> value == null ? 0 : value + 1 ); - // Compute the core partition count as well as the max partition num - Map intervalToCorePartition = new HashMap<>(); - Map intervalToPartitionNum = new HashMap<>(); - for (DataSegment segment : toInsertSegments) { - intervalToCorePartition.put(segment.getInterval(), segment.getShardSpec().getNumCorePartitions()); - intervalToPartitionNum.putIfAbsent(segment.getInterval(), 0); - int maxPartitionNum = Integer.max( - intervalToPartitionNum.get(segment.getInterval()), - segment.getShardSpec().getPartitionNum() - ); - intervalToPartitionNum.put(segment.getInterval(), maxPartitionNum); - } - - final List>> forwardSegmentsBatch = Lists.partition( - new ArrayList<>(segmentsToBeForwarded.entrySet()), - MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE - ); - for (List> batch : forwardSegmentsBatch) { - Map batchMap = new HashMap<>(); - for (Map.Entry entry : batch) { - batchMap.put(entry.getKey(), entry.getValue()); - } - // Find the segment metadata for the segment ids in this batch - List oldSegments = retrieveSegmentsById(handle, batchMap.keySet()); - for (DataSegment oldSegment : oldSegments) { - Interval newInterval = oldSegment.getInterval(); - for (DataSegment segment : toInsertSegments) { - // Check for interval overlap between each segment inserted with the segment in the batch. - // Overlapping segments can only be carried forward if their intervals are enclosed - // by those of the existing segments. - if (segment.getInterval().overlaps(newInterval)) { - if (segment.getInterval().contains(newInterval)) { - newInterval = segment.getInterval(); - } else { - throw new ISE("Incompatible segment intervals for commit: [%s] and [%s].", - newInterval, - segment.getInterval() - ); - } - } - } - TaskLockInfo lock = batchMap.get(oldSegment.getId().toString()); - // Update states for generating unique shard specs - final int partitionNum = intervalToPartitionNum.get(newInterval) + 1; - final int numCorePartitions = intervalToCorePartition.get(newInterval); - ShardSpec shardSpec = new NumberedShardSpec(partitionNum, numCorePartitions); - intervalToPartitionNum.put(newInterval, partitionNum); - // Create the new segment metadata for the already appended segment using the same load spec - DataSegment newSegment = new DataSegment( - oldSegment.getDataSource(), - newInterval, - lock.getVersion(), - oldSegment.getLoadSpec(), - oldSegment.getDimensions(), - oldSegment.getMetrics(), - shardSpec, - oldSegment.getBinaryVersion(), - oldSegment.getSize() - ); - retVal.add(newSegment); - final String now = DateTimes.nowUtc().toString(); - preparedBatch.add() - .bind("id", newSegment.getId().toString()) - .bind("dataSource", newSegment.getDataSource()) - .bind("created_date", now) - .bind("start", newSegment.getInterval().getStart().toString()) - .bind("end", newSegment.getInterval().getEnd().toString()) - .bind("partitioned", (newSegment.getShardSpec() instanceof NoneShardSpec) ? false : true) - .bind("version", newSegment.getVersion()) - .bind("used", true) - .bind("payload", jsonMapper.writeValueAsBytes(newSegment)) - .bind("used_status_last_updated", now); - } - final int[] affectedInsertRows = preparedBatch.execute(); + final int numCorePartitions = intervalToNumCorePartitions.get(newInterval); + ShardSpec shardSpec = new NumberedShardSpec(partitionNum, numCorePartitions); - final boolean succeeded = Arrays.stream(affectedInsertRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); - if (succeeded) { - log.info("Published segments with updated metadata to DB"); - } else { - throw new ISE("Failed to update segment metadatas in DB"); - } - } - } - catch (Exception e) { - log.errorSegments(segments, "Exception inserting segment metadata"); - throw e; + String lockVersion = carryForwardSegmentToLockVersion.get(oldSegment.getId().toString()); + DataSegment newSegment = DataSegment.builder(oldSegment) + .interval(newInterval) + .version(lockVersion) + .shardSpec(shardSpec) + .build(); + segmentsToInsert.add(newSegment); } - return retVal; + return segmentsToInsert; } /** @@ -1840,7 +1749,7 @@ private void verifySegmentsToCommit(Collection segments) * Callers of this method might need to retry as INSERT followed by SELECT * might fail due to race conditions. */ - private Set insertSegments(Set segments, Handle handle) + private Set insertSegments(Handle handle, Set segments) throws IOException { // Do not insert segment IDs which already exist @@ -1898,8 +1807,8 @@ private Set insertSegments(Set segments, Handle handle * {@link #MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE}. */ private void insertSegmentLockVersions( - Map segmentToReplaceLock, - Handle handle + Handle handle, + Map segmentToReplaceLock ) { if (segmentToReplaceLock.isEmpty()) { @@ -1908,9 +1817,9 @@ private void insertSegmentLockVersions( final PreparedBatch batch = handle.prepareBatch( StringUtils.format( - "INSERT INTO %1$s (id, dataSource, start, %2$send%2$s, segment_id, lock_version) " - + "VALUES (:id, :dataSource, :start, :end, :segment_id, :lock_version)", - dbTables.getSegmentVersionsTable(), connector.getQuoteString() + "INSERT INTO %1$s (id, group_id, segment_id, lock_version)" + + " VALUES (:id, :group_id, :segment_id, :lock_version)", + dbTables.getSegmentVersionsTable() ) ); @@ -1923,10 +1832,7 @@ private void insertSegmentLockVersions( DataSegment segment = entry.getKey(); TaskLockInfo lock = entry.getValue(); batch.add() - .bind("id", segment.getId() + ":" + lock.hashCode()) - .bind("dataSource", segment.getDataSource()) - .bind("start", lock.getInterval().getStartMillis()) - .bind("end", lock.getInterval().getEndMillis()) + .bind("group_id", lock.getGroupId()) .bind("segment_id", segment.getId().toString()) .bind("lock_version", lock.getVersion()); } @@ -1949,18 +1855,22 @@ private void insertSegmentLockVersions( private List retrieveSegmentsById(Handle handle, Set segmentIds) { - final String segmentIdCsv = segmentIds.stream().map(id -> "'" + id + "'") + final String segmentIdCsv = segmentIds.stream() + .map(id -> "'" + id + "'") .collect(Collectors.joining(",")); - final Query> query = handle.createQuery( - StringUtils.format( - "SELECT payload FROM %s WHERE id in (%s)", - dbTables.getSegmentsTable(), segmentIdCsv + ResultIterator resultIterator = handle + .createQuery( + StringUtils.format( + "SELECT payload FROM %s WHERE id in (%s)", + dbTables.getSegmentsTable(), segmentIdCsv + ) ) - ).setFetchSize(connector.getStreamingFetchSize()); - - ResultIterator resultIterator = query.map( - (index, r, ctx) -> JacksonUtils.readValue(jsonMapper, r.getBytes(1), DataSegment.class) - ).iterator(); + .setFetchSize(connector.getStreamingFetchSize()) + .map( + (index, r, ctx) -> + JacksonUtils.readValue(jsonMapper, r.getBytes(1), DataSegment.class) + ) + .iterator(); return Lists.newArrayList(resultIterator); } @@ -1977,66 +1887,38 @@ private String buildSqlToInsertSegments() ); } + /** + * Finds the append segments that were covered by the given task REPLACE locks. + * These append segments must now be carried forward to the same version as + * the segments being committed by this replace task. + * + * @return Map from append Segment ID to REPLACE lock version + */ @VisibleForTesting - Map getAppendedSegmentIds( + Map getAppendSegmentsCommittedDuringTask( Handle handle, - String datasource, Set replaceLocks ) { if (CollectionUtils.isNullOrEmpty(replaceLocks)) { return Collections.emptyMap(); } - final StringBuilder sb = new StringBuilder(); - sb.append( - StringUtils.format( - "SELECT segment_id, start, %1$send%1$s, lock_version FROM %2$s where dataSource = :dataSource AND (", - connector.getQuoteString(), dbTables.getSegmentVersionsTable() - ) - ); - List locks = new ArrayList<>(replaceLocks); - int n = locks.size(); - for (int i = 0; i < n; i++) { - sb.append( - StringUtils.format( - "(start = %2$s AND %1$send%1$s = %3$s AND lock_version = %4$s)", - connector.getQuoteString(), - StringUtils.format(":start%d", i), - StringUtils.format(":end%d", i), - StringUtils.format(":lock_version%d", i) - ) - ); - if (i < n - 1) { - sb.append(" OR "); - } - } - - sb.append(")"); - - Query> query = handle - .createQuery( - sb.toString() + final String groupId = replaceLocks.iterator().next().getGroupId(); + ResultIterator> resultIterator = handle + .createQuery("SELECT segment_id, lock_version FROM %2$s where group_id = :group_id") + .bind("group_id", groupId) + .map( + (index, r, ctx) -> Pair.of(r.getString("segment_id"), r.getString("lock_version")) ) - .bind("dataSource", datasource); - for (int i = 0; i < n; i++) { - query.bind(StringUtils.format("start%d", i), locks.get(i).getInterval().getStartMillis()) - .bind(StringUtils.format("end%d", i), locks.get(i).getInterval().getEndMillis()) - .bind(StringUtils.format("lock_version%d", i), locks.get(i).getVersion()); - } - - final ResultIterator> resultIterator = query.map((index, r, ctx) -> { - String segmentId = r.getString("segment_id"); - Interval interval = Intervals.utc(r.getLong("start"), r.getLong("end")); - String version = r.getString("lock_version"); - return Pair.of(segmentId, new TaskLockInfo(interval, version)); - }).iterator(); - Map retVal = new HashMap<>(); + .iterator(); + + final Map segmentIdToLockVersion = new HashMap<>(); while (resultIterator.hasNext()) { - Pair result = resultIterator.next(); - retVal.put(result.lhs, result.rhs); + Pair result = resultIterator.next(); + segmentIdToLockVersion.put(result.lhs, result.rhs); } - return retVal; + return segmentIdToLockVersion; } private Set segmentExistsBatch(final Handle handle, final Set segments) diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index d54d73090acf..250ea720c230 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -105,13 +105,18 @@ public String getCollation() } /** - * Auto-incrementing SQL type to use for IDs - * Must be an integer type, which values will be automatically set by the database - *

- * The resulting string will be interpolated into the table creation statement, e.g. - * CREATE TABLE druid_table ( id NOT NULL, ... ) + * Auto-incrementing integer SQL type to use for IDs. + * The returned string is interpolated into the table creation statement as follows: + *

+   * CREATE TABLE druid_table (
+   *   id <serial-type> NOT NULL,
+   *   col_2 VARCHAR(255) NOT NULL,
+   *   col_3 VARCHAR(255) NOT NULL
+   *   ...
+   * )
+   * 
* - * @return String representing the SQL type and auto-increment statement + * @return String representing auto-incrementing SQL integer type to use for IDs. */ public abstract String getSerialType(); @@ -342,20 +347,17 @@ private void createSegmentVersionTable(final String tableName) ImmutableList.of( StringUtils.format( "CREATE TABLE %1$s (\n" - + " id VARCHAR(255) NOT NULL,\n" - + " dataSource VARCHAR(255) %3$s NOT NULL,\n" - + " start BIGINT NOT NULL,\n" - + " %2$send%2$s BIGINT NOT NULL,\n" + + " id %2$s NOT NULL,\n" + + " group_id VARCHAR(255) NOT NULL,\n" + " segment_id VARCHAR(255) NOT NULL,\n" + " lock_version VARCHAR(255) NOT NULL,\n" + " PRIMARY KEY (id)\n" + ")", - tableName, getQuoteString(), getCollation() + tableName, getSerialType() ), StringUtils.format( - "CREATE INDEX idx_%1$s_datasource_end_start ON %1$s(dataSource, %2$send%2$s, start)", - tableName, - getQuoteString() + "CREATE INDEX idx_%1$s_group_lock_version ON %1$s(group_id, lock_version)", + tableName ) ) ); diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 23f03d1a4169..d90e9240373e 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -61,7 +61,6 @@ import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.PreparedBatch; import org.skife.jdbi.v2.util.StringMapper; @@ -87,9 +86,6 @@ public class IndexerSQLMetadataStorageCoordinatorTest @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - @Rule - public final ExpectedException expectedException = ExpectedException.none(); - private final ObjectMapper mapper = TestHelper.makeJsonMapper(); private final DataSegment defaultSegment = new DataSegment( @@ -468,12 +464,10 @@ private Boolean insertUsedSegments(Set dataSegments) ); } - private Map getAppendedSegmentIds(String datasource, Set replaceLocks) + private Map getAppendedSegmentIds(Set replaceLocks) { return derbyConnector.retryWithHandle( - handle -> { - return coordinator.getAppendedSegmentIds(handle, datasource, replaceLocks); - } + handle -> coordinator.getAppendSegmentsCommittedDuringTask(handle, replaceLocks) ); } @@ -635,7 +629,7 @@ public void testAllocateNewSegmentIds() segmentsToBeProcessed.addAll(month1); segmentsToBeProcessed.addAll(month2); final Map> segmentToNewIds = derbyConnector.retryWithHandle( - handle -> coordinator.allocateExtraIdsForAppendSegments(handle, "foo", segmentsToBeProcessed) + handle -> coordinator.getSegmentsToUpgradeOnAppend(handle, "foo", segmentsToBeProcessed) ); for (DataSegment segment : day1) { @@ -672,7 +666,7 @@ public void testCommitAppendSegments() { final Set allSegments = new HashSet<>(); final Set segmentIdsToBeCarriedForward = new HashSet<>(); - final TaskLockInfo lock = new TaskLockInfo(Intervals.of("2023-01-01/2023-01-03"), "2024-01-01"); + final TaskLockInfo lock = new TaskLockInfo("g1", Intervals.of("2023-01-01/2023-01-03"), "2024-01-01"); final Map segmentLockMap = new HashMap<>(); for (int i = 0; i < 10; i++) { @@ -732,11 +726,11 @@ public void testCommitAppendSegments() ); final Set replaceLocks = Collections.singleton(lock); - final Map segmentLockMetadata = getAppendedSegmentIds("foo", replaceLocks); + final Map segmentLockMetadata = getAppendedSegmentIds(replaceLocks); Assert.assertEquals(segmentIdsToBeCarriedForward, segmentLockMetadata.keySet()); Assert.assertEquals( - lock, - Iterables.getOnlyElement(ImmutableSet.copyOf(segmentLockMetadata.values())) + lock.getVersion(), + Iterables.getOnlyElement(segmentLockMetadata.values()) ); } @@ -744,7 +738,7 @@ public void testCommitAppendSegments() @Test public void testCommitReplaceSegments() { - final TaskLockInfo replaceLock = new TaskLockInfo(Intervals.of("2023-01-01/2023-02-01"), "2023-02-01"); + final TaskLockInfo replaceLock = new TaskLockInfo("g1", Intervals.of("2023-01-01/2023-02-01"), "2023-02-01"); final Set segmentsAppendedWithReplaceLock = new HashSet<>(); final Map appendedSegmentToReplaceLockMap = new HashMap<>(); for (int i = 1; i < 9; i++) { From a1c22a87c4b160d8d4bb750d6b2ec74e706ceb4a Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 3 Sep 2023 20:14:50 +0530 Subject: [PATCH 23/43] Get tests running --- .../task/ConcurrentReplaceAndAppendTest.java | 192 ++++++++++-------- .../task/batch/parallel/AppendTask.java | 12 +- .../task/batch/parallel/ReplaceTask.java | 11 +- .../IndexerSQLMetadataStorageCoordinator.java | 15 +- 4 files changed, 135 insertions(+), 95 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java index f810c3b81754..c40ff688177e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java @@ -42,6 +42,7 @@ import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.metadata.EntryExistsException; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; @@ -50,27 +51,28 @@ import org.apache.druid.tasklogs.NoopTaskLogs; import org.apache.druid.timeline.DataSegment; import org.joda.time.Period; +import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; -import java.util.TreeSet; -import java.util.function.Function; -import java.util.stream.Collectors; public class ConcurrentReplaceAndAppendTest extends IngestionTestBase { private static final Map REPLACE_CONTEXT = ImmutableMap.of(Tasks.TASK_LOCK_TYPE, "REPLACE"); private static final Map APPEND_CONTEXT = ImmutableMap.of(Tasks.TASK_LOCK_TYPE, "APPEND"); + private static final WorkerConfig WORKER_CONFIG = new WorkerConfig().setCapacity(10); private TaskQueue taskQueue; + private TaskRunner taskRunner; private TaskActionClient taskActionClient; - - private static final WorkerConfig WORKER_CONFIG = new WorkerConfig().setCapacity(10); + private final List runningTasks = new ArrayList<>(); @Before public void setup() @@ -79,7 +81,7 @@ public void setup() final TaskActionClientFactory taskActionClientFactory = createActionClientFactory(); taskActionClient = taskActionClientFactory.create(NoopTask.create()); final TaskToolboxFactory toolboxFactory = new TestTaskToolboxFactory(taskConfig, taskActionClientFactory); - final TaskRunner taskRunner = new ThreadingTaskRunner( + taskRunner = new ThreadingTaskRunner( toolboxFactory, taskConfig, WORKER_CONFIG, @@ -99,16 +101,45 @@ public void setup() taskActionClientFactory, getLockbox(), new NoopServiceEmitter() - ); + ) + { + @Override + public boolean add(Task task) throws EntryExistsException + { + boolean added = super.add(task); + if (added) { + runningTasks.add(task); + } + return added; + } + }; + runningTasks.clear(); taskQueue.start(); } + @After + public void tearDown() + { + for (Task task : runningTasks) { + if (task instanceof AppendTask) { + AppendTask appendTask = (AppendTask) task; + appendTask.markReady(); + appendTask.beginPublish(); + appendTask.completeSegmentAllocation(); + } else if (task instanceof ReplaceTask) { + ReplaceTask replaceTask = (ReplaceTask) task; + replaceTask.markReady(); + replaceTask.beginPublish(); + } + } + } + @Test public void testCommandExecutingTask() throws Exception { CommandExecutingTask replaceTask0 = new CommandExecutingTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023/2024"), Granularities.YEAR, REPLACE_CONTEXT, @@ -126,7 +157,7 @@ public void testCommandExecutingTask() throws Exception CommandExecutingTask appendTask0 = new CommandExecutingTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023/2024"), Granularities.YEAR, APPEND_CONTEXT, @@ -148,7 +179,7 @@ public void testCommandExecutingTask() throws Exception CommandExecutingTask replaceTask1 = new CommandExecutingTask( "replace1", - "DS", + DS.WIKI, Intervals.of("2023/2024"), Granularities.YEAR, REPLACE_CONTEXT, @@ -166,7 +197,7 @@ public void testCommandExecutingTask() throws Exception CommandExecutingTask replaceTask2 = new CommandExecutingTask( "replace2", - "DS", + DS.WIKI, Intervals.of("2023/2024"), Granularities.YEAR, REPLACE_CONTEXT, @@ -225,7 +256,7 @@ public void test() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023/2024"), Granularities.YEAR, null, @@ -234,7 +265,7 @@ public void test() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023/2024"), Granularities.YEAR, null @@ -242,7 +273,7 @@ public void test() throws Exception ReplaceTask replaceTask1 = new ReplaceTask( "replace1", - "DS", + DS.WIKI, Intervals.of("2023/2024"), Granularities.YEAR, null, @@ -251,7 +282,7 @@ public void test() throws Exception ReplaceTask replaceTask2 = new ReplaceTask( "replace2", - "DS", + DS.WIKI, Intervals.of("2023/2024"), Granularities.YEAR, null, @@ -301,7 +332,7 @@ public void testRRAA_dailyReplaceDailyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null, @@ -310,7 +341,7 @@ public void testRRAA_dailyReplaceDailyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null @@ -345,7 +376,7 @@ public void testRAAR_dailyReplaceDailyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null, @@ -354,7 +385,7 @@ public void testRAAR_dailyReplaceDailyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null @@ -390,7 +421,7 @@ public void testRARA_dailyReplaceDailyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null, @@ -399,7 +430,7 @@ public void testRARA_dailyReplaceDailyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null @@ -409,7 +440,6 @@ public void testRARA_dailyReplaceDailyAppend() throws Exception replaceTask0.markReady(); replaceTask0.awaitReadyComplete(); - taskQueue.add(appendTask0); appendTask0.markReady(); appendTask0.awaitReadyComplete(); @@ -437,7 +467,7 @@ public void testARRA_dailyReplaceDailyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null, @@ -446,7 +476,7 @@ public void testARRA_dailyReplaceDailyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null @@ -481,7 +511,7 @@ public void testARAR_dailyReplaceDailyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null, @@ -490,7 +520,7 @@ public void testARAR_dailyReplaceDailyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null @@ -527,7 +557,7 @@ public void testAARR_dailyReplaceDailyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null, @@ -536,7 +566,7 @@ public void testAARR_dailyReplaceDailyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null @@ -566,13 +596,12 @@ public void testAARR_dailyReplaceDailyAppend() throws Exception } - @Test public void testRRAA_monthlyReplaceDailyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, null, @@ -581,7 +610,7 @@ public void testRRAA_monthlyReplaceDailyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null @@ -616,7 +645,7 @@ public void testRAAR_monthlyReplaceDailyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, null, @@ -625,7 +654,7 @@ public void testRAAR_monthlyReplaceDailyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null @@ -661,7 +690,7 @@ public void testRARA_monthlyReplaceDailyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, null, @@ -670,7 +699,7 @@ public void testRARA_monthlyReplaceDailyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null @@ -708,7 +737,7 @@ public void testARRA_monthlyReplaceDailyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, null, @@ -717,7 +746,7 @@ public void testARRA_monthlyReplaceDailyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null @@ -752,7 +781,7 @@ public void testARAR_monthlyReplaceDailyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, null, @@ -761,7 +790,7 @@ public void testARAR_monthlyReplaceDailyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null @@ -798,7 +827,7 @@ public void testAARR_monthlyReplaceDailyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, null, @@ -807,7 +836,7 @@ public void testAARR_monthlyReplaceDailyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null @@ -837,13 +866,12 @@ public void testAARR_monthlyReplaceDailyAppend() throws Exception } - @Test public void testRRAA_dailyReplaceMonthlyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null, @@ -852,7 +880,7 @@ public void testRRAA_dailyReplaceMonthlyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, null @@ -887,7 +915,7 @@ public void testRAAR_dailyReplaceMonthlyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null, @@ -896,7 +924,7 @@ public void testRAAR_dailyReplaceMonthlyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, null @@ -932,7 +960,7 @@ public void testRARA_dailyReplaceMonthlyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null, @@ -941,7 +969,7 @@ public void testRARA_dailyReplaceMonthlyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, null @@ -979,7 +1007,7 @@ public void testARRA_dailyReplaceMonthlyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null, @@ -988,7 +1016,7 @@ public void testARRA_dailyReplaceMonthlyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, null @@ -1023,7 +1051,7 @@ public void testARAR_dailyReplaceMonthlyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null, @@ -1032,7 +1060,7 @@ public void testARAR_dailyReplaceMonthlyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, null @@ -1069,7 +1097,7 @@ public void testAARR_dailyReplaceMonthlyAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null, @@ -1078,7 +1106,7 @@ public void testAARR_dailyReplaceMonthlyAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.MONTH, null @@ -1112,7 +1140,7 @@ public void testMultipleAppend() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023/2024"), Granularities.YEAR, null, @@ -1121,7 +1149,7 @@ public void testMultipleAppend() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023/2024"), Granularities.YEAR, null @@ -1129,7 +1157,7 @@ public void testMultipleAppend() throws Exception AppendTask appendTask1 = new AppendTask( "append1", - "DS", + DS.WIKI, Intervals.of("2023/2024"), Granularities.YEAR, null @@ -1172,7 +1200,7 @@ public void testMultipleGranularities() throws Exception { ReplaceTask replaceTask0 = new ReplaceTask( "replace0", - "DS", + DS.WIKI, Intervals.of("2023/2024"), Granularities.YEAR, null, @@ -1181,7 +1209,7 @@ public void testMultipleGranularities() throws Exception AppendTask appendTask0 = new AppendTask( "append0", - "DS", + DS.WIKI, Intervals.of("2023-01-01/2023-02-01"), Granularities.DAY, null @@ -1189,7 +1217,7 @@ public void testMultipleGranularities() throws Exception AppendTask appendTask1 = new AppendTask( "append1", - "DS", + DS.WIKI, Intervals.of("2023-07-01/2024-01-01"), Granularities.QUARTER, null @@ -1197,7 +1225,7 @@ public void testMultipleGranularities() throws Exception AppendTask appendTask2 = new AppendTask( "append2", - "DS", + DS.WIKI, Intervals.of("2023-12-01/2024-01-01"), Granularities.MONTH, null @@ -1280,23 +1308,28 @@ private void verifyTaskFailure(Task task) private void verifySegmentCount(int expectedTotal, int expectedVisible) throws Exception { Collection allUsed = taskActionClient.submit( - new RetrieveUsedSegmentsAction("DS", null, ImmutableList.of(Intervals.ETERNITY), Segments.INCLUDING_OVERSHADOWED) + new RetrieveUsedSegmentsAction( + DS.WIKI, + null, + ImmutableList.of(Intervals.ETERNITY), + Segments.INCLUDING_OVERSHADOWED + ) ); - System.out.println("All used segments: " + allUsed.size()); - System.out.println(new TreeSet<>(allUsed.stream().map(s -> s.getId().toString()).collect(Collectors.toSet()))); + Assert.assertEquals(expectedTotal, allUsed.size()); + Collection visibleUsed = taskActionClient.submit( - new RetrieveUsedSegmentsAction("DS", null, ImmutableList.of(Intervals.ETERNITY), Segments.ONLY_VISIBLE) + new RetrieveUsedSegmentsAction( + DS.WIKI, + null, + ImmutableList.of(Intervals.ETERNITY), + Segments.ONLY_VISIBLE + ) ); - Assert.assertEquals(expectedTotal, allUsed.size()); - System.out.println("All visible segments: " + visibleUsed.size()); - System.out.println(new TreeSet<>(visibleUsed.stream().map(s -> s.getId().toString()).collect(Collectors.toSet()))); Assert.assertEquals(expectedVisible, visibleUsed.size()); } private class TestTaskToolboxFactory extends TaskToolboxFactory { - private final TaskConfig taskConfig; - public TestTaskToolboxFactory(TaskConfig taskConfig, TaskActionClientFactory taskActionClientFactory) { super( @@ -1339,26 +1372,17 @@ public TestTaskToolboxFactory(TaskConfig taskConfig, TaskActionClientFactory tas null, null ); - this.taskConfig = taskConfig; - } - - @Override - public TaskToolbox build(Task task) - { - return build(taskConfig, task); } - @Override - public TaskToolbox build(Function decoratorFn, Task task) - { - return build(decoratorFn.apply(taskConfig), task); - } - - @Override public TaskToolbox build(TaskConfig config, Task task) { return createTaskToolbox(config, task); } } + + private static class DS + { + static final String WIKI = "wiki"; + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java index db67ebab4b0b..42a9737890d3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java @@ -34,6 +34,7 @@ import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.AbstractTask; +import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -57,6 +58,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; +import java.util.stream.Collectors; /** * A test Task which mimics an appending task by having similar interactions with the overlord. @@ -156,10 +158,12 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception runLatch.await(); - if (publishSegments(toolbox, convertPendingSegments(pendingSegments))) { + SegmentPublishResult publishResult = publishSegments(toolbox, convertPendingSegments(pendingSegments)); + if (publishResult.isSuccess()) { return TaskStatus.success(getId()); + } else { + return TaskStatus.failure(getId(), publishResult.getErrorMsg()); } - return TaskStatus.failure(getId(), "Failed to append segments"); } @Override @@ -226,7 +230,7 @@ private Set convertPendingSegments(Set pend return segments; } - private boolean publishSegments(TaskToolbox toolbox, Set newSegments) + private SegmentPublishResult publishSegments(TaskToolbox toolbox, Set newSegments) throws Exception { final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> @@ -238,7 +242,7 @@ private boolean publishSegments(TaskToolbox toolbox, Set newSegment newSegments, Function.identity(), null - ).isSuccess(); + ); } @Override diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java index 485a1dc6cec6..2b8ee4b0a863 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java @@ -33,6 +33,7 @@ import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.AbstractTask; +import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; @@ -169,10 +170,12 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception runLatch.await(); final Set newSegments = createSegments(); - if (publishSegments(toolbox, oldSegments, newSegments)) { + final SegmentPublishResult publishResult = publishSegments(toolbox, oldSegments, newSegments); + if (publishResult.isSuccess()) { return TaskStatus.success(getId()); + } else { + return TaskStatus.failure(getId(), publishResult.getErrorMsg()); } - return TaskStatus.failure(getId(), "Failed to replace segments"); } @Override @@ -207,7 +210,7 @@ private Set createSegments() return newSegments; } - private boolean publishSegments(TaskToolbox toolbox, Set oldSegments, Set newSegments) + private SegmentPublishResult publishSegments(TaskToolbox toolbox, Set oldSegments, Set newSegments) throws Exception { final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> @@ -219,7 +222,7 @@ private boolean publishSegments(TaskToolbox toolbox, Set oldSegment newSegments, Function.identity(), null - ).isSuccess(); + ); } @Override diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 6a6229c26b6c..722e68b14934 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1817,8 +1817,8 @@ private void insertSegmentLockVersions( final PreparedBatch batch = handle.prepareBatch( StringUtils.format( - "INSERT INTO %1$s (id, group_id, segment_id, lock_version)" - + " VALUES (:id, :group_id, :segment_id, :lock_version)", + "INSERT INTO %1$s (group_id, segment_id, lock_version)" + + " VALUES (:group_id, :segment_id, :lock_version)", dbTables.getSegmentVersionsTable() ) ); @@ -1855,6 +1855,10 @@ private void insertSegmentLockVersions( private List retrieveSegmentsById(Handle handle, Set segmentIds) { + if (segmentIds.isEmpty()) { + return Collections.emptyList(); + } + final String segmentIdCsv = segmentIds.stream() .map(id -> "'" + id + "'") .collect(Collectors.joining(",")); @@ -1904,9 +1908,14 @@ Map getAppendSegmentsCommittedDuringTask( return Collections.emptyMap(); } + final String sql = StringUtils.format( + "SELECT segment_id, lock_version FROM %1$s WHERE group_id = :group_id", + dbTables.getSegmentVersionsTable() + ); + final String groupId = replaceLocks.iterator().next().getGroupId(); ResultIterator> resultIterator = handle - .createQuery("SELECT segment_id, lock_version FROM %2$s where group_id = :group_id") + .createQuery(sql) .bind("group_id", groupId) .map( (index, r, ctx) -> Pair.of(r.getString("segment_id"), r.getString("lock_version")) From 7b0e259cae115bbd85916e085a421a61bb4b7e76 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 3 Sep 2023 20:46:18 +0530 Subject: [PATCH 24/43] Fix IndexerStorageCoordinatorTest --- .../task/batch/parallel/AppendTask.java | 1 - ...exerSQLMetadataStorageCoordinatorTest.java | 32 +++++++++---------- 2 files changed, 16 insertions(+), 17 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java index 42a9737890d3..a8c293e72ea7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java @@ -58,7 +58,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; -import java.util.stream.Collectors; /** * A test Task which mimics an appending task by having similar interactions with the overlord. diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index d90e9240373e..8533f5cefa1b 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -479,10 +479,9 @@ private Boolean insertIntoSegmentVersionsTable(Map se PreparedBatch preparedBatch = handle.prepareBatch( StringUtils.format( StringUtils.format( - "INSERT INTO %1$s (id, dataSource, start, %2$send%2$s, segment_id, lock_version) " - + "VALUES (:id, :dataSource, :start, :end, :segment_id, :lock_version)", - table, - derbyConnector.getQuoteString() + "INSERT INTO %1$s (group_id, segment_id, lock_version) " + + "VALUES (:group_id, :segment_id, :lock_version)", + table ) ) ); @@ -490,10 +489,7 @@ private Boolean insertIntoSegmentVersionsTable(Map se final DataSegment segment = entry.getKey(); final TaskLockInfo lock = entry.getValue(); preparedBatch.add() - .bind("id", segment.getId() + ":" + lock.hashCode()) - .bind("dataSource", segment.getDataSource()) - .bind("start", lock.getInterval().getStartMillis()) - .bind("end", lock.getInterval().getEndMillis()) + .bind("group_id", lock.getGroupId()) .bind("segment_id", segment.getId().toString()) .bind("lock_version", lock.getVersion()); } @@ -560,6 +556,7 @@ public void testAllocateNewSegmentIds() ); day3.add(segment); } + final Set month2 = new HashSet<>(); for (int i = 0; i < 10; i++) { final DataSegment segment = new DataSegment( @@ -634,17 +631,21 @@ public void testAllocateNewSegmentIds() for (DataSegment segment : day1) { final Set newIds = segmentToNewIds.get(segment); - Assert.assertEquals(2, newIds.size()); + + // TODO: this test originally expected 2 new IDs but older version would already be overshadowed + Assert.assertEquals(1, newIds.size()); Assert.assertEquals( - ImmutableSet.of(v1, v2), + ImmutableSet.of(v2), newIds.stream().map(SegmentIdWithShardSpec::getVersion).collect(Collectors.toSet()) ); } for (DataSegment segment : day2) { final Set newIds = segmentToNewIds.get(segment); - Assert.assertEquals(2, newIds.size()); + + // TODO: this test originally expected 2 new IDs but older version would already be overshadowed + Assert.assertEquals(1, newIds.size()); Assert.assertEquals( - ImmutableSet.of(v1, v2), + ImmutableSet.of(v2), newIds.stream().map(SegmentIdWithShardSpec::getVersion).collect(Collectors.toSet()) ); } @@ -728,10 +729,9 @@ public void testCommitAppendSegments() final Set replaceLocks = Collections.singleton(lock); final Map segmentLockMetadata = getAppendedSegmentIds(replaceLocks); Assert.assertEquals(segmentIdsToBeCarriedForward, segmentLockMetadata.keySet()); - Assert.assertEquals( - lock.getVersion(), - Iterables.getOnlyElement(segmentLockMetadata.values()) - ); + + final Set lockVersions = new HashSet<>(segmentLockMetadata.values()); + Assert.assertEquals(lock.getVersion(), Iterables.getOnlyElement(lockVersions)); } From 2fd2b9ec27061ae8bb7524ece0c572f184fd6b91 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 4 Sep 2023 07:22:04 +0530 Subject: [PATCH 25/43] Fix for upstream changes --- .../common/actions/CommitMetadataAction.java | 127 ------------------ .../SegmentTransactionalAppendAction.java | 6 +- .../SegmentTransactionalReplaceAction.java | 6 +- .../indexing/common/actions/TaskAction.java | 1 - 4 files changed, 6 insertions(+), 134 deletions(-) delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitMetadataAction.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitMetadataAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitMetadataAction.java deleted file mode 100644 index 608a424b5979..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/CommitMetadataAction.java +++ /dev/null @@ -1,127 +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.actions; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.type.TypeReference; -import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.DataSourceMetadata; -import org.apache.druid.indexing.overlord.SegmentPublishResult; - -import javax.annotation.Nullable; - -/** - * A stream ingestion task didn't ingest any rows and created no segments (e.g., all records were unparseable), - * but still needs to update metadata with the progress that the task made. - */ -public class CommitMetadataAction implements TaskAction -{ - @Nullable - private final DataSourceMetadata startMetadata; - @Nullable - private final DataSourceMetadata endMetadata; - @Nullable - private final String dataSource; - - public static CommitMetadataAction create( - String dataSource, - DataSourceMetadata startMetadata, - DataSourceMetadata endMetadata - ) - { - return new CommitMetadataAction(startMetadata, endMetadata, dataSource); - } - - @JsonCreator - private CommitMetadataAction( - @JsonProperty("startMetadata") @Nullable DataSourceMetadata startMetadata, - @JsonProperty("endMetadata") @Nullable DataSourceMetadata endMetadata, - @JsonProperty("dataSource") @Nullable String dataSource - ) - { - this.startMetadata = startMetadata; - this.endMetadata = endMetadata; - this.dataSource = dataSource; - } - - @JsonProperty - @Nullable - public DataSourceMetadata getStartMetadata() - { - return startMetadata; - } - - @JsonProperty - @Nullable - public DataSourceMetadata getEndMetadata() - { - return endMetadata; - } - - @JsonProperty - @Nullable - public String getDataSource() - { - return dataSource; - } - - @Override - public TypeReference getReturnTypeReference() - { - return new TypeReference() - { - }; - } - - /** - * Performs some sanity checks and publishes the given segments. - */ - @Override - public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) - { - try { - return toolbox.getIndexerMetadataStorageCoordinator().commitMetadataOnly( - dataSource, - startMetadata, - endMetadata - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - @Override - public boolean isAudited() - { - return true; - } - - @Override - public String toString() - { - return "CommitMetadataAction{" + - "startMetadata=" + startMetadata + - ", endMetadata=" + endMetadata + - ", dataSource='" + dataSource + - '}'; - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java index 52496eac7782..adfd160a9b31 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java @@ -114,13 +114,13 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) IndexTaskUtils.setTaskDimensions(metricBuilder, task); if (retVal.isSuccess()) { - toolbox.getEmitter().emit(metricBuilder.build("segment/txn/success", 1)); + toolbox.getEmitter().emit(metricBuilder.setMetric("segment/txn/success", 1)); for (DataSegment segment : retVal.getSegments()) { IndexTaskUtils.setSegmentDimensions(metricBuilder, segment); - toolbox.getEmitter().emit(metricBuilder.build("segment/added/bytes", segment.getSize())); + toolbox.getEmitter().emit(metricBuilder.setMetric("segment/added/bytes", segment.getSize())); } } else { - toolbox.getEmitter().emit(metricBuilder.build("segment/txn/failure", 1)); + toolbox.getEmitter().emit(metricBuilder.setMetric("segment/txn/failure", 1)); } return retVal; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index 232951d7eee6..bccbe3278e16 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -116,16 +116,16 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) IndexTaskUtils.setTaskDimensions(metricBuilder, task); if (retVal.isSuccess()) { - toolbox.getEmitter().emit(metricBuilder.build("segment/txn/success", 1)); + toolbox.getEmitter().emit(metricBuilder.setMetric("segment/txn/success", 1)); for (DataSegment segment : retVal.getSegments()) { final String partitionType = segment.getShardSpec() == null ? null : segment.getShardSpec().getType(); metricBuilder.setDimension(DruidMetrics.PARTITIONING_TYPE, partitionType); metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); - toolbox.getEmitter().emit(metricBuilder.build("segment/added/bytes", segment.getSize())); + toolbox.getEmitter().emit(metricBuilder.setMetric("segment/added/bytes", segment.getSize())); } } else { - toolbox.getEmitter().emit(metricBuilder.build("segment/txn/failure", 1)); + toolbox.getEmitter().emit(metricBuilder.setMetric("segment/txn/failure", 1)); } return retVal; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java index 82833c5ea64f..171d53b9cdd6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java @@ -36,7 +36,6 @@ @JsonSubTypes.Type(name = "lockRelease", value = LockReleaseAction.class), @JsonSubTypes.Type(name = "segmentInsertion", value = SegmentInsertAction.class), @JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class), - @JsonSubTypes.Type(name = "commitMetadata", value = SegmentTransactionalInsertAction.class), @JsonSubTypes.Type(name = "segmentTransactionalAppend", value = SegmentTransactionalAppendAction.class), @JsonSubTypes.Type(name = "segmentTransactionalReplace", value = SegmentTransactionalReplaceAction.class), // Type name doesn't correspond to the name of the class for backward compatibility. From 55eca9090b14bd4d426256a4371f6b0272b42b8e Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 4 Sep 2023 10:08:01 +0530 Subject: [PATCH 26/43] Use supervisorTaskId inside ReplaceTaskLock --- .../SegmentTransactionalAppendAction.java | 4 +- .../SegmentTransactionalReplaceAction.java | 6 +- .../indexing/common/actions/TaskLocks.java | 33 ++----- .../druid/indexing/overlord/TaskLockbox.java | 86 ++++++++++++++----- ...TestIndexerMetadataStorageCoordinator.java | 6 +- .../IndexerMetadataStorageCoordinator.java | 4 +- ...TaskLockInfo.java => ReplaceTaskLock.java} | 24 ++++-- .../IndexerSQLMetadataStorageCoordinator.java | 32 +++---- .../druid/metadata/SQLMetadataConnector.java | 4 +- ...exerSQLMetadataStorageCoordinatorTest.java | 26 +++--- 10 files changed, 126 insertions(+), 99 deletions(-) rename server/src/main/java/org/apache/druid/indexing/overlord/{TaskLockInfo.java => ReplaceTaskLock.java} (68%) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java index adfd160a9b31..9b90ce61f2b2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java @@ -25,8 +25,8 @@ import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; +import org.apache.druid.indexing.overlord.ReplaceTaskLock; import org.apache.druid.indexing.overlord.SegmentPublishResult; -import org.apache.druid.indexing.overlord.TaskLockInfo; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; @@ -81,7 +81,7 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); final String datasource = task.getDataSource(); - final Map segmentToReplaceLock + final Map segmentToReplaceLock = TaskLocks.findReplaceLocksCoveringSegments(datasource, toolbox.getTaskLockbox(), segments); final SegmentPublishResult retVal; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index bccbe3278e16..6dd2fd9dbd6f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -26,8 +26,8 @@ import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; +import org.apache.druid.indexing.overlord.ReplaceTaskLock; import org.apache.druid.indexing.overlord.SegmentPublishResult; -import org.apache.druid.indexing.overlord.TaskLockInfo; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.SegmentUtils; @@ -85,8 +85,8 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); // Find the active replace locks held only by this task - final Set replaceLocksForTask - = TaskLocks.findReplaceLocksHeldByTask(task, toolbox.getTaskLockbox()); + final Set replaceLocksForTask + = toolbox.getTaskLockbox().findReplaceLocksForTask(task); final SegmentPublishResult retVal; try { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java index 439345c3d5f9..bf16e2369d1e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java @@ -26,7 +26,7 @@ import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TimeChunkLock; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.TaskLockInfo; +import org.apache.druid.indexing.overlord.ReplaceTaskLock; import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.java.util.common.ISE; import org.apache.druid.timeline.DataSegment; @@ -43,7 +43,6 @@ import java.util.NavigableMap; import java.util.Set; import java.util.TreeMap; -import java.util.stream.Collectors; public class TaskLocks { @@ -125,7 +124,7 @@ public static boolean isLockCoversSegments( * @return Map from segment to REPLACE lock that completely covers it. The map * does not contain an entry for segments that have no covering REPLACE lock. */ - public static Map findReplaceLocksCoveringSegments( + public static Map findReplaceLocksCoveringSegments( final String datasource, final TaskLockbox taskLockbox, final Set segments @@ -139,16 +138,12 @@ public static Map findReplaceLocksCoveringSegments( ).add(segment) ); - final Set replaceLocks = taskLockbox.getAllReplaceLocksForDatasource(datasource) - .stream() - .map(TaskLocks::toLockInfo) - .collect(Collectors.toSet()); - - final Map segmentToReplaceLock = new HashMap<>(); + final Set replaceLocks = taskLockbox.getAllReplaceLocksForDatasource(datasource); + final Map segmentToReplaceLock = new HashMap<>(); intervalToSegments.forEach((interval, segmentsInInterval) -> { // For each interval, find the lock that covers it, if any - for (TaskLockInfo lock : replaceLocks) { + for (ReplaceTaskLock lock : replaceLocks) { if (lock.getInterval().contains(interval)) { segmentsInInterval.forEach(s -> segmentToReplaceLock.put(s, lock)); return; @@ -159,24 +154,6 @@ public static Map findReplaceLocksCoveringSegments( return segmentToReplaceLock; } - /** - * Finds the active locks of type {@link TaskLockType#REPLACE} held by this task. - */ - public static Set findReplaceLocksHeldByTask(Task task, TaskLockbox taskLockbox) - { - return taskLockbox - .findLocksForTask(task) - .stream() - .filter(taskLock -> !taskLock.isRevoked() && TaskLockType.REPLACE.equals(taskLock.getType())) - .map(TaskLocks::toLockInfo) - .collect(Collectors.toSet()); - } - - public static TaskLockInfo toLockInfo(TaskLock taskLock) - { - return new TaskLockInfo(taskLock.getGroupId(), taskLock.getInterval(), taskLock.getVersion()); - } - public static List findLocksForSegments( final Task task, final TaskLockbox taskLockbox, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 63304deb9d25..2eb9ce047f08 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -886,6 +886,71 @@ public List findLocksForTask(final Task task) } } + /** + * Finds the active non-revoked REPLACE locks held by the given task. + */ + public Set findReplaceLocksForTask(Task task) + { + giant.lock(); + try { + return getNonRevokedReplaceLocks(findLockPossesForTask(task), task.getDataSource()); + } + finally { + giant.unlock(); + } + } + + /** + * Finds all the active non-revoked REPLACE locks for the given datasource. + */ + public Set getAllReplaceLocksForDatasource(String datasource) + { + giant.lock(); + try { + final NavigableMap>> activeLocks = running.get(datasource); + if (activeLocks == null) { + return ImmutableSet.of(); + } + + List lockPosses + = activeLocks.values() + .stream() + .flatMap(map -> map.values().stream()) + .flatMap(Collection::stream) + .collect(Collectors.toList()); + return getNonRevokedReplaceLocks(lockPosses, datasource); + } + finally { + giant.unlock(); + } + } + + private Set getNonRevokedReplaceLocks(List posses, String datasource) + { + final Set replaceLocks = new HashSet<>(); + for (TaskLockPosse posse : posses) { + final TaskLock lock = posse.getTaskLock(); + if (lock.isRevoked() || !TaskLockType.REPLACE.equals(posse.getTaskLock().getType())) { + continue; + } + + // Replace locks are always held by the supervisor task + if (posse.taskIds.size() > 1) { + throw new ISE( + "Replace lock[%s] for datasource[%s] is held by multiple tasks[%s]", + lock, datasource, posse.taskIds + ); + } + + String supervisorTaskId = posse.taskIds.iterator().next(); + replaceLocks.add( + new ReplaceTaskLock(supervisorTaskId, lock.getInterval(), lock.getVersion()) + ); + } + + return replaceLocks; + } + /** * Gets a List of Intervals locked by higher priority tasks for each datasource. * Here, Segment Locks are being treated the same as Time Chunk Locks i.e. @@ -1242,27 +1307,6 @@ Map>>> ge return running; } - public Set getAllReplaceLocksForDatasource(final String datasource) - { - giant.lock(); - try { - final NavigableMap>> activeLocks = running.get(datasource); - if (activeLocks == null) { - return ImmutableSet.of(); - } - return activeLocks.values() - .stream() - .flatMap(map -> map.values().stream()) - .flatMap(Collection::stream) - .map(TaskLockPosse::getTaskLock) - .filter(taskLock -> !taskLock.isRevoked() && taskLock.getType().equals(TaskLockType.REPLACE)) - .collect(Collectors.toSet()); - } - finally { - giant.unlock(); - } - } - /** * Check if the lock for a given request can coexist with a given set of conflicting posses without any revocation. diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index 42995c84bf9c..6d788e6c23d0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -25,10 +25,10 @@ import com.google.common.collect.Sets; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.ReplaceTaskLock; import org.apache.druid.indexing.overlord.SegmentCreateRequest; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; -import org.apache.druid.indexing.overlord.TaskLockInfo; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Pair; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; @@ -160,7 +160,7 @@ public Map allocatePendingSegments @Override public SegmentPublishResult commitReplaceSegments( Set replaceSegments, - Set locksHeldByReplaceTask + Set locksHeldByReplaceTask ) { // Don't actually compare metadata, just do it! @@ -170,7 +170,7 @@ public SegmentPublishResult commitReplaceSegments( @Override public SegmentPublishResult commitAppendSegments( Set appendSegments, - Map appendSegmentToReplaceLock + Map appendSegmentToReplaceLock ) { return SegmentPublishResult.ok(commitSegments(appendSegments)); diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index cf0a095a2c7a..7f5bfedb906f 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -288,7 +288,7 @@ SegmentPublishResult commitSegmentsAndMetadata( */ SegmentPublishResult commitAppendSegments( Set appendSegments, - Map appendSegmentToReplaceLock + Map appendSegmentToReplaceLock ); /** @@ -299,7 +299,7 @@ SegmentPublishResult commitAppendSegments( */ SegmentPublishResult commitReplaceSegments( Set replaceSegments, - Set locksHeldByReplaceTask + Set locksHeldByReplaceTask ); /** diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/TaskLockInfo.java b/server/src/main/java/org/apache/druid/indexing/overlord/ReplaceTaskLock.java similarity index 68% rename from server/src/main/java/org/apache/druid/indexing/overlord/TaskLockInfo.java rename to server/src/main/java/org/apache/druid/indexing/overlord/ReplaceTaskLock.java index a7dcb0c68b96..c34d98377565 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/TaskLockInfo.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/ReplaceTaskLock.java @@ -23,22 +23,28 @@ import java.util.Objects; -public class TaskLockInfo +/** + * Details of a REPLACE lock held by a batch supervisor task. + *

+ * Replace locks are always held by the supervisor task, i.e. ParallelIndexSupervisorTask + * in case of native batch ingestion and ControllerTask in case of MSQ ingestion. + */ +public class ReplaceTaskLock { - private final String groupId; + private final String supervisorTaskId; private final Interval interval; private final String version; - public TaskLockInfo(String groupId, Interval interval, String version) + public ReplaceTaskLock(String supervisorTaskId, Interval interval, String version) { - this.groupId = groupId; + this.supervisorTaskId = supervisorTaskId; this.interval = interval; this.version = version; } - public String getGroupId() + public String getSupervisorTaskId() { - return groupId; + return supervisorTaskId; } public Interval getInterval() @@ -60,8 +66,8 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - TaskLockInfo that = (TaskLockInfo) o; - return Objects.equals(groupId, that.groupId) + ReplaceTaskLock that = (ReplaceTaskLock) o; + return Objects.equals(supervisorTaskId, that.supervisorTaskId) && Objects.equals(interval, that.interval) && Objects.equals(version, that.version); } @@ -69,6 +75,6 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(groupId, interval, version); + return Objects.hash(supervisorTaskId, interval, version); } } diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 722e68b14934..c88f114a7e52 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -35,10 +35,10 @@ import org.apache.commons.lang.StringEscapeUtils; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.ReplaceTaskLock; import org.apache.druid.indexing.overlord.SegmentCreateRequest; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; -import org.apache.druid.indexing.overlord.TaskLockInfo; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -409,7 +409,7 @@ public SegmentPublishResult inTransaction( @Override public SegmentPublishResult commitReplaceSegments( final Set replaceSegments, - final Set locksHeldByReplaceTask + final Set locksHeldByReplaceTask ) { verifySegmentsToCommit(replaceSegments); @@ -437,7 +437,7 @@ public SegmentPublishResult commitReplaceSegments( @Override public SegmentPublishResult commitAppendSegments( final Set appendSegments, - final Map appendSegmentToReplaceLock + final Map appendSegmentToReplaceLock ) { verifySegmentsToCommit(appendSegments); @@ -1661,7 +1661,7 @@ private Set announceHistoricalSegmentBatch( private Set getSegmentsToUpgradeOnReplace( final Handle handle, final Set replaceSegments, - final Set locksHeldByReplaceTask + final Set locksHeldByReplaceTask ) { // For each replace interval, find the number of core partitions and total partitions @@ -1808,7 +1808,7 @@ private Set insertSegments(Handle handle, Set segments */ private void insertSegmentLockVersions( Handle handle, - Map segmentToReplaceLock + Map segmentToReplaceLock ) { if (segmentToReplaceLock.isEmpty()) { @@ -1817,22 +1817,22 @@ private void insertSegmentLockVersions( final PreparedBatch batch = handle.prepareBatch( StringUtils.format( - "INSERT INTO %1$s (group_id, segment_id, lock_version)" - + " VALUES (:group_id, :segment_id, :lock_version)", + "INSERT INTO %1$s (task_id, segment_id, lock_version)" + + " VALUES (:task_id, :segment_id, :lock_version)", dbTables.getSegmentVersionsTable() ) ); - final List>> partitions = Lists.partition( + final List>> partitions = Lists.partition( new ArrayList<>(segmentToReplaceLock.entrySet()), MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE ); - for (List> partition : partitions) { - for (Map.Entry entry : partition) { + for (List> partition : partitions) { + for (Map.Entry entry : partition) { DataSegment segment = entry.getKey(); - TaskLockInfo lock = entry.getValue(); + ReplaceTaskLock lock = entry.getValue(); batch.add() - .bind("group_id", lock.getGroupId()) + .bind("task_id", lock.getSupervisorTaskId()) .bind("segment_id", segment.getId().toString()) .bind("lock_version", lock.getVersion()); } @@ -1901,7 +1901,7 @@ private String buildSqlToInsertSegments() @VisibleForTesting Map getAppendSegmentsCommittedDuringTask( Handle handle, - Set replaceLocks + Set replaceLocks ) { if (CollectionUtils.isNullOrEmpty(replaceLocks)) { @@ -1909,14 +1909,14 @@ Map getAppendSegmentsCommittedDuringTask( } final String sql = StringUtils.format( - "SELECT segment_id, lock_version FROM %1$s WHERE group_id = :group_id", + "SELECT segment_id, lock_version FROM %1$s WHERE task_id = :task_id", dbTables.getSegmentVersionsTable() ); - final String groupId = replaceLocks.iterator().next().getGroupId(); + final String groupId = replaceLocks.iterator().next().getSupervisorTaskId(); ResultIterator> resultIterator = handle .createQuery(sql) - .bind("group_id", groupId) + .bind("task_id", groupId) .map( (index, r, ctx) -> Pair.of(r.getString("segment_id"), r.getString("lock_version")) ) diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index 250ea720c230..69a81b1d1913 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -348,7 +348,7 @@ private void createSegmentVersionTable(final String tableName) StringUtils.format( "CREATE TABLE %1$s (\n" + " id %2$s NOT NULL,\n" - + " group_id VARCHAR(255) NOT NULL,\n" + + " task_id VARCHAR(255) NOT NULL,\n" + " segment_id VARCHAR(255) NOT NULL,\n" + " lock_version VARCHAR(255) NOT NULL,\n" + " PRIMARY KEY (id)\n" @@ -356,7 +356,7 @@ private void createSegmentVersionTable(final String tableName) tableName, getSerialType() ), StringUtils.format( - "CREATE INDEX idx_%1$s_group_lock_version ON %1$s(group_id, lock_version)", + "CREATE INDEX idx_%1$s_task ON %1$s(task_id)", tableName ) ) diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 8533f5cefa1b..2794381d6f10 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -27,10 +27,10 @@ import org.apache.druid.data.input.StringTuple; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.ObjectMetadata; +import org.apache.druid.indexing.overlord.ReplaceTaskLock; import org.apache.druid.indexing.overlord.SegmentCreateRequest; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; -import org.apache.druid.indexing.overlord.TaskLockInfo; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; @@ -464,14 +464,14 @@ private Boolean insertUsedSegments(Set dataSegments) ); } - private Map getAppendedSegmentIds(Set replaceLocks) + private Map getAppendedSegmentIds(Set replaceLocks) { return derbyConnector.retryWithHandle( handle -> coordinator.getAppendSegmentsCommittedDuringTask(handle, replaceLocks) ); } - private Boolean insertIntoSegmentVersionsTable(Map segmentToTaskLockMap) + private Boolean insertIntoSegmentVersionsTable(Map segmentToTaskLockMap) { final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentVersionsTable(); return derbyConnector.retryWithHandle( @@ -479,17 +479,17 @@ private Boolean insertIntoSegmentVersionsTable(Map se PreparedBatch preparedBatch = handle.prepareBatch( StringUtils.format( StringUtils.format( - "INSERT INTO %1$s (group_id, segment_id, lock_version) " - + "VALUES (:group_id, :segment_id, :lock_version)", + "INSERT INTO %1$s (task_id, segment_id, lock_version) " + + "VALUES (:task_id, :segment_id, :lock_version)", table ) ) ); - for (Map.Entry entry : segmentToTaskLockMap.entrySet()) { + for (Map.Entry entry : segmentToTaskLockMap.entrySet()) { final DataSegment segment = entry.getKey(); - final TaskLockInfo lock = entry.getValue(); + final ReplaceTaskLock lock = entry.getValue(); preparedBatch.add() - .bind("group_id", lock.getGroupId()) + .bind("task_id", lock.getSupervisorTaskId()) .bind("segment_id", segment.getId().toString()) .bind("lock_version", lock.getVersion()); } @@ -667,8 +667,8 @@ public void testCommitAppendSegments() { final Set allSegments = new HashSet<>(); final Set segmentIdsToBeCarriedForward = new HashSet<>(); - final TaskLockInfo lock = new TaskLockInfo("g1", Intervals.of("2023-01-01/2023-01-03"), "2024-01-01"); - final Map segmentLockMap = new HashMap<>(); + final ReplaceTaskLock lock = new ReplaceTaskLock("g1", Intervals.of("2023-01-01/2023-01-03"), "2024-01-01"); + final Map segmentLockMap = new HashMap<>(); for (int i = 0; i < 10; i++) { final DataSegment segment = new DataSegment( @@ -726,7 +726,7 @@ public void testCommitAppendSegments() ImmutableSet.copyOf(retrieveUsedSegmentIds()) ); - final Set replaceLocks = Collections.singleton(lock); + final Set replaceLocks = Collections.singleton(lock); final Map segmentLockMetadata = getAppendedSegmentIds(replaceLocks); Assert.assertEquals(segmentIdsToBeCarriedForward, segmentLockMetadata.keySet()); @@ -738,9 +738,9 @@ public void testCommitAppendSegments() @Test public void testCommitReplaceSegments() { - final TaskLockInfo replaceLock = new TaskLockInfo("g1", Intervals.of("2023-01-01/2023-02-01"), "2023-02-01"); + final ReplaceTaskLock replaceLock = new ReplaceTaskLock("g1", Intervals.of("2023-01-01/2023-02-01"), "2023-02-01"); final Set segmentsAppendedWithReplaceLock = new HashSet<>(); - final Map appendedSegmentToReplaceLockMap = new HashMap<>(); + final Map appendedSegmentToReplaceLockMap = new HashMap<>(); for (int i = 1; i < 9; i++) { final DataSegment segment = new DataSegment( "foo", From 1ba0e8dda82beaefda2331041b961cd9c513fca8 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 6 Sep 2023 06:48:54 +0530 Subject: [PATCH 27/43] Add CommandQueueTask, remove unneeded classes --- .../common/task/CommandExecutingTask.java | 332 ----- .../common/task/CommandQueueTask.java | 171 +++ .../task/ConcurrentReplaceAndAppendTest.java | 1292 ++--------------- .../task/batch/parallel/ActionsTestTask.java | 77 + .../task/batch/parallel/AppendTask.java | 278 ---- .../task/batch/parallel/ReplaceTask.java | 253 ---- .../IndexerSQLMetadataStorageCoordinator.java | 95 +- ...exerSQLMetadataStorageCoordinatorTest.java | 14 +- 8 files changed, 415 insertions(+), 2097 deletions(-) delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandExecutingTask.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandQueueTask.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ActionsTestTask.java delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandExecutingTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandExecutingTask.java deleted file mode 100644 index 07386bd7d75c..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandExecutingTask.java +++ /dev/null @@ -1,332 +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.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import org.apache.druid.indexer.TaskStatus; -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.TaskToolbox; -import org.apache.druid.indexing.common.actions.SegmentAllocateAction; -import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; -import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; -import org.apache.druid.indexing.common.actions.SurrogateTaskActionClient; -import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; -import org.apache.druid.indexing.common.config.TaskConfig; -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.granularity.Granularities; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; -import org.apache.druid.server.security.ResourceAction; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.partition.NumberedPartialShardSpec; -import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.apache.druid.timeline.partition.ShardSpec; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -import javax.annotation.Nonnull; -import java.io.IOException; -import java.util.Collections; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; - -/** - * A test Task which can run a Runnable for its isReady and runTask methods - */ -public class CommandExecutingTask extends AbstractTask -{ - private final Interval interval; - private final Granularity segmentGranularity; - private final TaskLockType lockType; - private final int priority; - private final CountDownLatch readyLatch = new CountDownLatch(1); - private final CountDownLatch readyComplete = new CountDownLatch(1); - private final CountDownLatch runLatch = new CountDownLatch(1); - private final CountDownLatch runComplete = new CountDownLatch(1); - private final CountDownLatch publishLatch = new CountDownLatch(1); - private String version; - private Runnable command; - - private final Set pendingSegments = new HashSet<>(); - private final AtomicInteger sequenceId = new AtomicInteger(0); - - private TaskToolbox toolbox; - - public CommandExecutingTask( - String id, - String dataSource, - Interval interval, - Granularity segmentGranularity, - Map context, - IngestionMode ingestionMode - ) - { - super( - id == null ? StringUtils.format("command_%s_%s", DateTimes.nowUtc(), UUID.randomUUID().toString()) : id, - dataSource == null ? "none" : dataSource, - context, - ingestionMode - ); - this.interval = interval; - this.segmentGranularity = segmentGranularity; - this.lockType = TaskLockType.valueOf(getContextValue(Tasks.TASK_LOCK_TYPE, "EXCLUSIVE")); - this.priority = getContextValue(Tasks.PRIORITY_KEY, 0); - } - - @Override - public String getType() - { - return "command"; - } - - @Nonnull - @Override - public Set getInputSourceResources() - { - return ImmutableSet.of(); - } - - @Override - public String setup(TaskToolbox toolbox) throws Exception - { - readyLatch.await(); - while (!isReady(toolbox.getTaskActionClient())) { - Thread.sleep(100); - } - return null; - } - - @Override - public boolean isReady(TaskActionClient taskActionClient) throws Exception - { - return tryTimeChunkLockSingleInterval( - new SurrogateTaskActionClient(getId(), taskActionClient), - interval, - lockType - ); - } - - private boolean tryTimeChunkLockSingleInterval(TaskActionClient client, Interval interval, TaskLockType lockType) - throws IOException - { - final TaskLock lock = client.submit(new TimeChunkLockTryAcquireAction(lockType, interval)); - if (lock == null) { - return false; - } - if (lock.isRevoked()) { - throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", interval)); - } - version = lock.getVersion(); - return true; - } - - @Override - public void stopGracefully(TaskConfig taskConfig) - { - } - - @Override - public TaskStatus runTask(TaskToolbox toolbox) throws Exception - { - this.toolbox = toolbox; - - readyComplete.countDown(); - - runLatch.await(); - - try { - command.run(); - return TaskStatus.success(getId()); - } - catch (Exception e) { - return TaskStatus.failure(getId(), "Failed to execute the command."); - } - } - - @Override - public void cleanUp(TaskToolbox toolbox, TaskStatus taskStatus) throws Exception - { - super.cleanUp(toolbox, taskStatus); - runComplete.countDown(); - } - - @Override - public int getPriority() - { - return priority; - } - - public void awaitRunComplete() throws InterruptedException - { - runComplete.await(); - } - - public void awaitReadyComplete() throws InterruptedException - { - readyComplete.await(); - } - - public void markReady() - { - readyLatch.countDown(); - } - - public void beginPublish() - { - publishLatch.countDown(); - } - - public void runCommand(Runnable command) - { - this.command = command; - runLatch.countDown(); - } - - public Set appendSegments(Set newSegments) throws Exception - { - publishLatch.await(); - if (!TaskLockType.APPEND.equals(lockType)) { - throw new ISE("appendSegments can be called only with an APPEND lock."); - } - final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> - toolbox.getTaskActionClient().submit( - SegmentTransactionalAppendAction.create(segmentsToPublish) - ); - return publisher.publishSegments( - Collections.emptySet(), - newSegments, - Function.identity(), - null - ).getSegments(); - } - - public Set convertPendingSegments(Set pendingSegments) - { - final Set segments = new HashSet<>(); - for (SegmentIdWithShardSpec pendingSegment : pendingSegments) { - final SegmentId id = pendingSegment.asSegmentId(); - segments.add( - new DataSegment( - id, - ImmutableMap.of(id.toString(), id.toString()), - ImmutableList.of(), - ImmutableList.of(), - pendingSegment.getShardSpec(), - null, - 0, - 0 - ) - ); - } - return segments; - } - - - public Set replaceSegments(Set newSegments) throws Exception - { - if (!TaskLockType.REPLACE.equals(lockType)) { - throw new ISE("replaceSegments can be called only with a REPLACEk lock."); - } - final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> - toolbox.getTaskActionClient().submit( - SegmentTransactionalReplaceAction.create(segmentsToPublish) - ); - return publisher.publishSegments( - null, - newSegments, - Function.identity(), - null - ).getSegments(); - } - - public Set createCorePartitions(int numSegmentsPerInterval) - { - final Set newSegments = new HashSet<>(); - for (int i = 0; i < numSegmentsPerInterval; i++) { - for (Interval subInterval : segmentGranularity.getIterable(interval)) { - final ShardSpec shardSpec = new NumberedShardSpec(i, numSegmentsPerInterval); - final SegmentId segmentId = SegmentId.of(getDataSource(), subInterval, version, shardSpec); - newSegments.add( - new DataSegment( - segmentId, - ImmutableMap.of(segmentId.toString(), segmentId.toString()), - ImmutableList.of(), - ImmutableList.of(), - shardSpec, - null, - 0, - 0 - ) - ); - } - } - return newSegments; - } - - public SegmentIdWithShardSpec allocateOrGetSegmentForTimestamp(String timestamp) - { - final DateTime time = DateTimes.of(timestamp); - for (SegmentIdWithShardSpec pendingSegment : pendingSegments) { - if (pendingSegment.getInterval().contains(time)) { - return pendingSegment; - } - } - return allocateNewSegmentForDate(time); - } - - private SegmentIdWithShardSpec allocateNewSegmentForDate(DateTime time) - { - try { - SegmentAllocateAction allocateAction = new SegmentAllocateAction( - getDataSource(), - time, - Granularities.NONE, - segmentGranularity, - getId() + "_" + sequenceId.getAndIncrement(), - null, - false, - NumberedPartialShardSpec.instance(), - LockGranularity.TIME_CHUNK, - lockType - ); - final SegmentIdWithShardSpec id = toolbox.getTaskActionClient().submit(allocateAction); - pendingSegments.add(id); - return id; - } - catch (Exception e) { - return null; - } - - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandQueueTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandQueueTask.java new file mode 100644 index 000000000000..8f0aefc49d76 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandQueueTask.java @@ -0,0 +1,171 @@ +package org.apache.druid.indexing.common.task; + +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.config.TaskConfig; +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.logger.Logger; + +import java.util.UUID; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Test task that can be given a series of commands to execute in its {@link #runTask} method. + */ +public class CommandQueueTask extends AbstractTask +{ + private static final Logger log = new Logger(CommandQueueTask.class); + + private final BlockingQueue> commandQueue = new LinkedBlockingQueue<>(); + private final AtomicBoolean finishRequested = new AtomicBoolean(false); + private final AtomicInteger numCommandsExecuted = new AtomicInteger(0); + + private final AtomicReference finalTaskStatus = new AtomicReference<>(); + + public CommandQueueTask(String datasource, String groupId) + { + super( + StringUtils.format("test_%s_%s", DateTimes.nowUtc(), UUID.randomUUID().toString()), + groupId, + null, + datasource, + null + ); + } + + /** + * Marks the run of this task as finished so that no new commands are accepted. + * This methods waits for all the commands submitted so far to finish execution + * and returns the final TaskStatus. + */ + public TaskStatus finishRunAndGetStatus() + { + // Mark finished to prevent submission of any more commands + finishRequested.set(true); + + // Submit a dummy command to ensure that all previous commands have finished + executeInternal(() -> 1); + + return finalTaskStatus.get(); + } + + /** + * Submits the given runnable for execution on the task thread. This method + * returns immediately and does not wait for the execution to finish. + */ + public void submit(Runnable runnable) + { + if (finishRequested.get()) { + throw new ISE("Task[%s] cannot accept any more commands as it is already shutting down.", getId()); + } + + // Add a command with a dummy return value + Command command = new Command<>( + () -> { + runnable.run(); + return 1; + } + ); + commandQueue.offer(command); + } + + /** + * Executes the given callable on the task thread. This method waits until the + * execution has finished and returns the computed value. + */ + public V execute(Callable callable) + { + if (finishRequested.get()) { + throw new ISE("Task[%s] cannot accept any more commands as it is already shutting down.", getId()); + } + + return executeInternal(callable); + } + + private V executeInternal(Callable callable) + { + Command command = new Command<>(callable); + commandQueue.offer(command); + + try { + return command.value.get(10, TimeUnit.SECONDS); + } + catch (Exception e) { + throw new ISE(e, "Error waiting for command on task[%s] to finish", getId()); + } + } + + @Override + public TaskStatus runTask(TaskToolbox taskToolbox) throws Exception + { + TaskStatus status; + try { + while (commandQueue.size() > 0 || !finishRequested.get()) { + Command command = commandQueue.poll(10, TimeUnit.SECONDS); + if (command != null) { + command.execute(); + numCommandsExecuted.incrementAndGet(); + } + } + status = TaskStatus.success(getId()); + } + catch (Exception e) { + log.error(e, "Error while running command[%d] for task[%s]", numCommandsExecuted.get(), getId()); + status = TaskStatus.failure(getId(), e.getMessage()); + } + + finalTaskStatus.set(status); + return status; + } + + @Override + public String getType() + { + return "test_command_executing"; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + return true; + } + + @Override + public void stopGracefully(TaskConfig taskConfig) + { + + } + + private static class Command + { + final Callable callable; + final CompletableFuture value = new CompletableFuture<>(); + + Command(Callable callable) + { + this.callable = callable; + } + + void execute() throws Exception + { + try { + V result = callable.call(); + value.complete(result); + } + catch (Exception e) { + value.completeExceptionally(e); + throw e; + } + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java index c40ff688177e..96c7467a4ec0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java @@ -20,8 +20,9 @@ package org.apache.druid.indexing.common.task; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Sets; import org.apache.druid.indexing.common.MultipleFileTaskReportFileWriter; +import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskStorageDirTracker; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolboxFactory; @@ -30,8 +31,7 @@ import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; -import org.apache.druid.indexing.common.task.batch.parallel.AppendTask; -import org.apache.druid.indexing.common.task.batch.parallel.ReplaceTask; +import org.apache.druid.indexing.common.task.batch.parallel.ActionsTestTask; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.overlord.TaskQueue; import org.apache.druid.indexing.overlord.TaskRunner; @@ -41,8 +41,6 @@ import org.apache.druid.indexing.overlord.config.TaskQueueConfig; import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.metadata.EntryExistsException; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; @@ -50,6 +48,8 @@ import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.tasklogs.NoopTaskLogs; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.joda.time.Interval; import org.joda.time.Period; import org.junit.After; import org.junit.Assert; @@ -58,28 +58,25 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.HashSet; +import java.util.Collections; import java.util.List; -import java.util.Map; -import java.util.Set; public class ConcurrentReplaceAndAppendTest extends IngestionTestBase { - private static final Map REPLACE_CONTEXT = ImmutableMap.of(Tasks.TASK_LOCK_TYPE, "REPLACE"); - private static final Map APPEND_CONTEXT = ImmutableMap.of(Tasks.TASK_LOCK_TYPE, "APPEND"); private static final WorkerConfig WORKER_CONFIG = new WorkerConfig().setCapacity(10); private TaskQueue taskQueue; private TaskRunner taskRunner; - private TaskActionClient taskActionClient; + private TaskActionClientFactory taskActionClientFactory; + private TaskActionClient dummyTaskActionClient; private final List runningTasks = new ArrayList<>(); @Before public void setup() { final TaskConfig taskConfig = new TaskConfigBuilder().build(); - final TaskActionClientFactory taskActionClientFactory = createActionClientFactory(); - taskActionClient = taskActionClientFactory.create(NoopTask.create()); + taskActionClientFactory = createActionClientFactory(); + dummyTaskActionClient = taskActionClientFactory.create(NoopTask.create()); final TaskToolboxFactory toolboxFactory = new TestTaskToolboxFactory(taskConfig, taskActionClientFactory); taskRunner = new ThreadingTaskRunner( toolboxFactory, @@ -101,18 +98,7 @@ public void setup() taskActionClientFactory, getLockbox(), new NoopServiceEmitter() - ) - { - @Override - public boolean add(Task task) throws EntryExistsException - { - boolean added = super.add(task); - if (added) { - runningTasks.add(task); - } - return added; - } - }; + ); runningTasks.clear(); taskQueue.start(); } @@ -121,1162 +107,91 @@ public boolean add(Task task) throws EntryExistsException public void tearDown() { for (Task task : runningTasks) { - if (task instanceof AppendTask) { - AppendTask appendTask = (AppendTask) task; - appendTask.markReady(); - appendTask.beginPublish(); - appendTask.completeSegmentAllocation(); - } else if (task instanceof ReplaceTask) { - ReplaceTask replaceTask = (ReplaceTask) task; - replaceTask.markReady(); - replaceTask.beginPublish(); - } - } - } - - @Test - public void testCommandExecutingTask() throws Exception - { - CommandExecutingTask replaceTask0 = new CommandExecutingTask( - "replace0", - DS.WIKI, - Intervals.of("2023/2024"), - Granularities.YEAR, - REPLACE_CONTEXT, - AbstractTask.IngestionMode.REPLACE - ); - Runnable runReplaceTask0 = () -> { - try { - final Set segments = replaceTask0.createCorePartitions(1); - replaceTask0.replaceSegments(segments); - } - catch (Exception e) { - throw new RuntimeException(e); - } - }; - - CommandExecutingTask appendTask0 = new CommandExecutingTask( - "append0", - DS.WIKI, - Intervals.of("2023/2024"), - Granularities.YEAR, - APPEND_CONTEXT, - AbstractTask.IngestionMode.APPEND - ); - Runnable runAppendTask0 = () -> { - try { - final Set pendingSegments = new HashSet<>(); - pendingSegments.add( - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-01") - ); - final Set segments = appendTask0.convertPendingSegments(pendingSegments); - appendTask0.appendSegments(segments); - } - catch (Exception e) { - throw new RuntimeException(e); - } - }; - - CommandExecutingTask replaceTask1 = new CommandExecutingTask( - "replace1", - DS.WIKI, - Intervals.of("2023/2024"), - Granularities.YEAR, - REPLACE_CONTEXT, - AbstractTask.IngestionMode.REPLACE - ); - Runnable runReplaceTask1 = () -> { - try { - final Set segments = replaceTask1.createCorePartitions(1); - replaceTask1.replaceSegments(segments); - } - catch (Exception e) { - throw new RuntimeException(e); - } - }; - - CommandExecutingTask replaceTask2 = new CommandExecutingTask( - "replace2", - DS.WIKI, - Intervals.of("2023/2024"), - Granularities.YEAR, - REPLACE_CONTEXT, - AbstractTask.IngestionMode.REPLACE - ); - Runnable runReplaceTask2 = () -> { - try { - final Set segments = replaceTask2.createCorePartitions(1); - replaceTask2.replaceSegments(segments); - } - catch (Exception e) { - throw new RuntimeException(e); + if (task instanceof ActionsTestTask) { + ((ActionsTestTask) task).finishRunAndGetStatus(); } - }; - - - // Create a set of initial segments - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.runCommand(runReplaceTask0); - replaceTask0.awaitRunComplete(); - verifySegmentCount(1, 1); - verifyTaskSuccess(replaceTask0); - - // Append task begins and allocates pending segments - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - appendTask0.runCommand(runAppendTask0); - - // New replace task starts and ends before the appending task finishes - taskQueue.add(replaceTask1); - replaceTask1.markReady(); - replaceTask1.runCommand(runReplaceTask1); - replaceTask1.awaitRunComplete(); - verifySegmentCount(2, 1); - verifyTaskSuccess(replaceTask1); - - taskQueue.add(replaceTask2); - replaceTask2.markReady(); - replaceTask2.awaitReadyComplete(); - - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(4, 2); - verifyTaskSuccess(appendTask0); - - replaceTask2.runCommand(runReplaceTask2); - replaceTask2.awaitRunComplete(); - verifySegmentCount(6, 2); - verifyTaskSuccess(replaceTask2); - } - - @Test - public void test() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023/2024"), - Granularities.YEAR, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023/2024"), - Granularities.YEAR, - null - ); - - ReplaceTask replaceTask1 = new ReplaceTask( - "replace1", - DS.WIKI, - Intervals.of("2023/2024"), - Granularities.YEAR, - null, - null - ); - - ReplaceTask replaceTask2 = new ReplaceTask( - "replace2", - DS.WIKI, - Intervals.of("2023/2024"), - Granularities.YEAR, - null, - null - ); - - // Create a set of initial segments - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(1, 1); - verifyTaskSuccess(replaceTask0); - - // Append task begins and allocates pending segments - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-01"); - appendTask0.completeSegmentAllocation(); - - // New replace task starts and ends before the appending task finishes - taskQueue.add(replaceTask1); - replaceTask1.markReady(); - replaceTask1.beginPublish(); - replaceTask1.awaitRunComplete(); - verifySegmentCount(2, 1); - verifyTaskSuccess(replaceTask1); - - taskQueue.add(replaceTask2); - replaceTask2.markReady(); - replaceTask2.awaitReadyComplete(); - - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(4, 2); - verifyTaskSuccess(appendTask0); - - replaceTask2.beginPublish(); - replaceTask2.awaitRunComplete(); - verifySegmentCount(6, 2); - verifyTaskSuccess(replaceTask2); - } - - @Test - public void testRRAA_dailyReplaceDailyAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null - ); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(31, 31); - verifyTaskSuccess(replaceTask0); - - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(62, 62); - verifyTaskSuccess(appendTask0); - } - - @Test - public void testRAAR_dailyReplaceDailyAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null - ); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.awaitReadyComplete(); - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(31, 31); - verifyTaskSuccess(appendTask0); - - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(93, 62); - verifyTaskSuccess(replaceTask0); - } - - @Test - public void testRARA_dailyReplaceDailyAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null - ); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.awaitReadyComplete(); - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(31, 31); - verifyTaskSuccess(replaceTask0); - - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(93, 62); - verifyTaskSuccess(appendTask0); } @Test - public void testARRA_dailyReplaceDailyAppend() throws Exception + public void testAppendSegmentGetsUpgraded() throws Exception { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null - ); - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(31, 31); - verifyTaskSuccess(replaceTask0); - - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(93, 62); - verifyTaskSuccess(appendTask0); - } - - @Test - public void testARAR_dailyReplaceDailyAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null - ); - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.awaitReadyComplete(); - - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(31, 31); - verifyTaskSuccess(appendTask0); - - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(93, 62); - verifyTaskSuccess(replaceTask0); + final Interval year2023 = Intervals.of("2023/2024"); + + // Commit initial segments for v0 + ActionsTestTask replaceTask0 = createAndStartTask(); + TaskLock replaceLock = replaceTask0.acquireReplaceLockOn(year2023); + + final DataSegment segmentV00 + = DataSegment.builder() + .dataSource(DS.WIKI) + .interval(year2023) + .version(replaceLock.getVersion()) + .size(1) + .build(); + + replaceTask0.commitReplaceSegments(segmentV00); + replaceTask0.finishRunAndGetStatus(); + verifyIntervalHasUsedSegments(year2023, segmentV00); + verifyIntervalHasVisibleSegments(year2023, segmentV00); + + // Allocate an append segment for v0 + final ActionsTestTask appendTask0 = createAndStartTask(); + appendTask0.acquireAppendLockOn(year2023); + // TODO: fix allocation and version of allocated segment + final SegmentIdWithShardSpec pendingSegmentV01 = appendTask0.allocateSegment(); + + // Commit replace segment for v1 + final ActionsTestTask replaceTask1 = createAndStartTask(); + replaceTask1.acquireReplaceLockOn(year2023); + + final DataSegment segmentV10 = DataSegment.builder(segmentV00).version("v1").build(); + replaceTask1.commitReplaceSegments(segmentV10); + replaceTask1.finishRunAndGetStatus(); + verifyIntervalHasUsedSegments(year2023, segmentV00, segmentV10); + verifyIntervalHasVisibleSegments(year2023, segmentV10); + + final ActionsTestTask replaceTask2 = createAndStartTask(); + replaceTask2.acquireReplaceLockOn(year2023); + + // Commit append segment v0 and verify that it gets upgraded to v1 + final DataSegment segmentV01 = asSegment(pendingSegmentV01); + appendTask0.commitAppendSegments(segmentV01); + appendTask0.finishRunAndGetStatus(); + + final DataSegment segmentV11 = DataSegment.builder(segmentV01).version(segmentV10.getVersion()).build(); + verifyIntervalHasUsedSegments( + year2023, + segmentV00, segmentV01, segmentV10, segmentV11 + ); + verifyIntervalHasVisibleSegments(year2023, segmentV10, segmentV11); + + // Commit replace segment v2 and verify that append segment gets upgraded to v2 + final DataSegment segmentV20 = DataSegment.builder(segmentV00).version("v2").build(); + replaceTask2.commitReplaceSegments(segmentV20); + replaceTask2.finishRunAndGetStatus(); + + final DataSegment segmentV21 = DataSegment.builder(segmentV01).version(segmentV20.getVersion()).build(); + verifyIntervalHasUsedSegments( + year2023, + segmentV00, segmentV01, segmentV10, segmentV11, segmentV20, segmentV21 + ); + verifyIntervalHasVisibleSegments(year2023, segmentV20, segmentV21); } - @Test - public void testAARR_dailyReplaceDailyAppend() throws Exception + private static DataSegment asSegment(SegmentIdWithShardSpec pendingSegment) { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, + final SegmentId id = pendingSegment.asSegmentId(); + return new DataSegment( + id, + Collections.singletonMap(id.toString(), id.toString()), + Collections.emptyList(), + Collections.emptyList(), + pendingSegment.getShardSpec(), null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null + 0, + 0 ); - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(31, 31); - verifyTaskSuccess(appendTask0); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(62, 31); - verifyTaskSuccess(replaceTask0); - } - - - @Test - public void testRRAA_monthlyReplaceDailyAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.MONTH, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null - ); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(1, 1); - verifyTaskSuccess(replaceTask0); - - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(2, 2); - verifyTaskSuccess(appendTask0); - } - - @Test - public void testRAAR_monthlyReplaceDailyAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.MONTH, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null - ); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.awaitReadyComplete(); - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(31, 31); - verifyTaskSuccess(appendTask0); - - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(63, 32); - verifyTaskSuccess(replaceTask0); - } - - @Test - public void testRARA_monthlyReplaceDailyAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.MONTH, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null - ); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.awaitReadyComplete(); - - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(1, 1); - verifyTaskSuccess(replaceTask0); - - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(63, 32); - verifyTaskSuccess(appendTask0); - } - - @Test - public void testARRA_monthlyReplaceDailyAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.MONTH, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null - ); - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(1, 1); - verifyTaskSuccess(replaceTask0); - - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(63, 32); - verifyTaskSuccess(appendTask0); - } - - @Test - public void testARAR_monthlyReplaceDailyAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.MONTH, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null - ); - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.awaitReadyComplete(); - - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(31, 31); - verifyTaskSuccess(appendTask0); - - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(63, 32); - verifyTaskSuccess(replaceTask0); - } - - @Test - public void testAARR_monthlyReplaceDailyAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.MONTH, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null - ); - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(31, 31); - verifyTaskSuccess(appendTask0); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(32, 1); - verifyTaskSuccess(replaceTask0); - } - - - @Test - public void testRRAA_dailyReplaceMonthlyAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.MONTH, - null - ); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(31, 31); - verifyTaskSuccess(replaceTask0); - - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(62, 62); - verifyTaskSuccess(appendTask0); - } - - @Test - public void testRAAR_dailyReplaceMonthlyAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.MONTH, - null - ); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.awaitReadyComplete(); - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(1, 1); - verifyTaskSuccess(appendTask0); - - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(1, 1); - verifyTaskFailure(replaceTask0); - } - - @Test - public void testRARA_dailyReplaceMonthlyAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.MONTH, - null - ); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.awaitReadyComplete(); - - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(31, 31); - verifyTaskSuccess(replaceTask0); - - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(31, 31); - verifyTaskFailure(appendTask0); - } - - @Test - public void testARRA_dailyReplaceMonthlyAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.MONTH, - null - ); - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(31, 31); - verifyTaskSuccess(replaceTask0); - - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(31, 31); - verifyTaskFailure(appendTask0); - } - - @Test - public void testARAR_dailyReplaceMonthlyAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.MONTH, - null - ); - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.awaitReadyComplete(); - - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(1, 1); - verifyTaskSuccess(appendTask0); - - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(1, 1); - verifyTaskFailure(replaceTask0); - } - - @Test - public void testAARR_dailyReplaceMonthlyAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.MONTH, - null - ); - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - for (int i = 1; i <= 9; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-0" + i); - } - for (int i = 10; i <= 31; i++) { - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-" + i); - } - appendTask0.completeSegmentAllocation(); - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(1, 1); - verifyTaskSuccess(appendTask0); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(32, 31); - verifyTaskSuccess(replaceTask0); - } - - @Test - public void testMultipleAppend() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023/2024"), - Granularities.YEAR, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023/2024"), - Granularities.YEAR, - null - ); - - AppendTask appendTask1 = new AppendTask( - "append1", - DS.WIKI, - Intervals.of("2023/2024"), - Granularities.YEAR, - null - ); - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-01"); - appendTask0.completeSegmentAllocation(); - - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.awaitReadyComplete(); - - taskQueue.add(appendTask1); - appendTask1.markReady(); - appendTask1.awaitReadyComplete(); - appendTask1.allocateOrGetSegmentForTimestamp("2023-01-01"); - appendTask1.completeSegmentAllocation(); - appendTask1.beginPublish(); - appendTask1.awaitRunComplete(); - verifySegmentCount(1, 1); - verifyTaskSuccess(appendTask1); - - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(3, 2); - verifyTaskSuccess(replaceTask0); - - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(5, 3); - verifyTaskSuccess(appendTask0); - } - - @Test - public void testMultipleGranularities() throws Exception - { - ReplaceTask replaceTask0 = new ReplaceTask( - "replace0", - DS.WIKI, - Intervals.of("2023/2024"), - Granularities.YEAR, - null, - null - ); - - AppendTask appendTask0 = new AppendTask( - "append0", - DS.WIKI, - Intervals.of("2023-01-01/2023-02-01"), - Granularities.DAY, - null - ); - - AppendTask appendTask1 = new AppendTask( - "append1", - DS.WIKI, - Intervals.of("2023-07-01/2024-01-01"), - Granularities.QUARTER, - null - ); - - AppendTask appendTask2 = new AppendTask( - "append2", - DS.WIKI, - Intervals.of("2023-12-01/2024-01-01"), - Granularities.MONTH, - null - ); - - taskQueue.add(appendTask0); - appendTask0.markReady(); - appendTask0.awaitReadyComplete(); - appendTask0.allocateOrGetSegmentForTimestamp("2023-01-01"); - appendTask0.completeSegmentAllocation(); - - - taskQueue.add(appendTask1); - appendTask1.markReady(); - appendTask1.awaitReadyComplete(); - appendTask1.allocateOrGetSegmentForTimestamp("2023-07-01"); - appendTask1.allocateOrGetSegmentForTimestamp("2023-08-01"); - appendTask1.allocateOrGetSegmentForTimestamp("2023-09-01"); - appendTask1.allocateOrGetSegmentForTimestamp("2023-10-01"); - appendTask1.allocateOrGetSegmentForTimestamp("2023-11-01"); - appendTask1.allocateOrGetSegmentForTimestamp("2023-12-01"); - appendTask1.completeSegmentAllocation(); - appendTask1.beginPublish(); - appendTask1.awaitRunComplete(); - verifySegmentCount(2, 2); - verifyTaskSuccess(appendTask1); - - taskQueue.add(appendTask2); - appendTask2.markReady(); - appendTask2.awaitReadyComplete(); - appendTask2.allocateOrGetSegmentForTimestamp("2023-12-01"); - appendTask2.completeSegmentAllocation(); - - taskQueue.add(replaceTask0); - replaceTask0.markReady(); - replaceTask0.awaitReadyComplete(); - - appendTask0.beginPublish(); - appendTask0.awaitRunComplete(); - verifySegmentCount(3, 3); - verifyTaskSuccess(appendTask0); - - replaceTask0.beginPublish(); - replaceTask0.awaitRunComplete(); - verifySegmentCount(5, 2); - verifyTaskSuccess(replaceTask0); - - appendTask2.beginPublish(); - appendTask2.awaitRunComplete(); - verifySegmentCount(7, 3); - verifyTaskSuccess(appendTask2); } private void verifyTaskSuccess(Task task) @@ -1305,27 +220,27 @@ private void verifyTaskFailure(Task task) Assert.assertTrue(getTaskStorage().getStatus(task.getId()).get().isFailure()); } - private void verifySegmentCount(int expectedTotal, int expectedVisible) throws Exception + private void verifyIntervalHasUsedSegments(Interval interval, DataSegment... expectedSegments) throws Exception { - Collection allUsed = taskActionClient.submit( - new RetrieveUsedSegmentsAction( - DS.WIKI, - null, - ImmutableList.of(Intervals.ETERNITY), - Segments.INCLUDING_OVERSHADOWED - ) - ); - Assert.assertEquals(expectedTotal, allUsed.size()); + verifySegments(interval, Segments.INCLUDING_OVERSHADOWED, expectedSegments); + } + + private void verifyIntervalHasVisibleSegments(Interval interval, DataSegment... expectedSegments) throws Exception + { + verifySegments(interval, Segments.ONLY_VISIBLE, expectedSegments); + } - Collection visibleUsed = taskActionClient.submit( + private void verifySegments(Interval interval, Segments visibility, DataSegment... expectedSegments) throws Exception + { + Collection allUsedSegments = dummyTaskActionClient.submit( new RetrieveUsedSegmentsAction( DS.WIKI, null, - ImmutableList.of(Intervals.ETERNITY), - Segments.ONLY_VISIBLE + ImmutableList.of(interval), + visibility ) ); - Assert.assertEquals(expectedVisible, visibleUsed.size()); + Assert.assertEquals(Sets.newHashSet(expectedSegments), Sets.newHashSet(allUsedSegments)); } private class TestTaskToolboxFactory extends TaskToolboxFactory @@ -1385,4 +300,13 @@ private static class DS { static final String WIKI = "wiki"; } + + private ActionsTestTask createAndStartTask() + { + ActionsTestTask task = new ActionsTestTask("wiki", taskActionClientFactory); + taskQueue.add(task); + runningTasks.add(task); + return task; + } + } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ActionsTestTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ActionsTestTask.java new file mode 100644 index 000000000000..07e67a87560b --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ActionsTestTask.java @@ -0,0 +1,77 @@ +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.google.common.collect.Sets; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; +import org.apache.druid.indexing.common.actions.TaskAction; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.actions.TaskActionClientFactory; +import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; +import org.apache.druid.indexing.common.task.CommandQueueTask; +import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; + +/** + * Test task that can only invoke task actions. + */ +public class ActionsTestTask extends CommandQueueTask +{ + private final TaskActionClient client; + + public ActionsTestTask(String datasource, TaskActionClientFactory factory) + { + super(datasource, null); + this.client = factory.create(this); + } + + public TaskLock acquireReplaceLockOn(Interval interval) + { + return tryTimeChunkLock(interval, TaskLockType.REPLACE); + } + + public TaskLock acquireAppendLockOn(Interval interval) + { + return tryTimeChunkLock(interval, TaskLockType.APPEND); + } + + public SegmentPublishResult commitReplaceSegments(DataSegment... segments) + { + return runAction( + SegmentTransactionalReplaceAction.create(Sets.newHashSet(segments)) + ); + } + + public SegmentPublishResult commitAppendSegments(DataSegment... segments) + { + return runAction( + SegmentTransactionalAppendAction.create(Sets.newHashSet(segments)) + ); + } + + public SegmentIdWithShardSpec allocateSegment() + { + return null; + } + + private TaskLock tryTimeChunkLock(Interval interval, TaskLockType lockType) + { + final TaskLock lock = runAction(new TimeChunkLockTryAcquireAction(lockType, interval)); + if (lock == null) { + throw new ISE("Could not acquire [%s] lock on interval[%s] for task[%s]", lockType, interval, getId()); + } else if (lock.isRevoked()) { + throw new ISE("Acquired [%s] lock on interval[%s] for task[%s] has been revoked.", lockType, interval, getId()); + } + + return lock; + } + + private T runAction(TaskAction action) + { + return execute(() -> client.submit(action)); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java deleted file mode 100644 index a8c293e72ea7..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AppendTask.java +++ /dev/null @@ -1,278 +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.batch.parallel; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import org.apache.druid.indexer.TaskStatus; -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.TaskToolbox; -import org.apache.druid.indexing.common.actions.SegmentAllocateAction; -import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; -import org.apache.druid.indexing.common.actions.SurrogateTaskActionClient; -import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; -import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.task.AbstractTask; -import org.apache.druid.indexing.overlord.SegmentPublishResult; -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.granularity.Granularities; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; -import org.apache.druid.server.security.ResourceAction; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.partition.NumberedPartialShardSpec; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -import javax.annotation.Nonnull; -import java.io.IOException; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; - -/** - * A test Task which mimics an appending task by having similar interactions with the overlord. - * - * Begins running by acquiring an APPEND lock and immediately allocates pending segments. - * - * Task ends after publishing these pending segments and relevant metadata entries in a transaction - */ -public class AppendTask extends AbstractTask -{ - private final Interval interval; - private final Granularity segmentGranularity; - private final TaskLockType lockType; - private final int priority; - private final CountDownLatch readyLatch = new CountDownLatch(1); - private final CountDownLatch runLatch = new CountDownLatch(1); - private final CountDownLatch segmentAllocationComplete = new CountDownLatch(1); - private final CountDownLatch runComplete = new CountDownLatch(1); - private final CountDownLatch readyComplete = new CountDownLatch(1); - - private final Set pendingSegments = new HashSet<>(); - - private TaskToolbox toolbox; - private final AtomicInteger sequenceId = new AtomicInteger(0); - - public AppendTask( - String id, - String dataSource, - Interval interval, - Granularity segmentGranularity, - Integer priority - ) - { - super( - id == null ? StringUtils.format("append_%s_%s", DateTimes.nowUtc(), UUID.randomUUID().toString()) : id, - dataSource == null ? "none" : dataSource, - null, - IngestionMode.APPEND - ); - this.interval = interval; - this.segmentGranularity = segmentGranularity; - this.lockType = TaskLockType.APPEND; - this.priority = priority == null ? 50 : priority; - } - - @Override - public String getType() - { - return "replace"; - } - - @Nonnull - @Override - public Set getInputSourceResources() - { - return ImmutableSet.of(); - } - - @Override - public boolean isReady(TaskActionClient taskActionClient) throws Exception - { - readyLatch.await(); - return tryTimeChunkLockSingleInterval( - new SurrogateTaskActionClient(getId(), taskActionClient), - interval, - lockType - ); - } - - private boolean tryTimeChunkLockSingleInterval(TaskActionClient client, Interval interval, TaskLockType lockType) - throws IOException - { - final TaskLock lock = client.submit(new TimeChunkLockTryAcquireAction(lockType, interval)); - if (lock == null) { - return false; - } - if (lock.isRevoked()) { - throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", interval)); - } - return true; - } - - @Override - public void stopGracefully(TaskConfig taskConfig) - { - } - - @Override - public TaskStatus runTask(TaskToolbox toolbox) throws Exception - { - this.toolbox = toolbox; - readyComplete.countDown(); - - //final Set pendingSegments = allocatePendingSegments(toolbox); - - segmentAllocationComplete.await(); - - runLatch.await(); - - SegmentPublishResult publishResult = publishSegments(toolbox, convertPendingSegments(pendingSegments)); - if (publishResult.isSuccess()) { - return TaskStatus.success(getId()); - } else { - return TaskStatus.failure(getId(), publishResult.getErrorMsg()); - } - } - - @Override - public void cleanUp(TaskToolbox toolbox, TaskStatus taskStatus) throws Exception - { - super.cleanUp(toolbox, taskStatus); - runComplete.countDown(); - } - - public SegmentIdWithShardSpec allocateOrGetSegmentForTimestamp(String timestamp) - { - final DateTime time = DateTimes.of(timestamp); - for (SegmentIdWithShardSpec pendingSegment : pendingSegments) { - if (pendingSegment.getInterval().contains(time)) { - return pendingSegment; - } - } - return allocateNewSegmentForDate(time); - } - - private SegmentIdWithShardSpec allocateNewSegmentForDate(DateTime time) - { - try { - SegmentAllocateAction allocateAction = new SegmentAllocateAction( - getDataSource(), - time, - Granularities.NONE, - segmentGranularity, - getId() + "_" + sequenceId.getAndIncrement(), - null, - false, - NumberedPartialShardSpec.instance(), - LockGranularity.TIME_CHUNK, - lockType - ); - final SegmentIdWithShardSpec id = toolbox.getTaskActionClient().submit(allocateAction); - pendingSegments.add(id); - return id; - } - catch (Exception e) { - return null; - } - - } - - private Set convertPendingSegments(Set pendingSegments) - { - final Set segments = new HashSet<>(); - for (SegmentIdWithShardSpec pendingSegment : pendingSegments) { - final SegmentId id = pendingSegment.asSegmentId(); - segments.add( - new DataSegment( - id, - ImmutableMap.of(id.toString(), id.toString()), - ImmutableList.of(), - ImmutableList.of(), - pendingSegment.getShardSpec(), - null, - 0, - 0 - ) - ); - } - return segments; - } - - private SegmentPublishResult publishSegments(TaskToolbox toolbox, Set newSegments) - throws Exception - { - final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> - toolbox.getTaskActionClient().submit( - SegmentTransactionalAppendAction.create(segmentsToPublish) - ); - return publisher.publishSegments( - Collections.emptySet(), - newSegments, - Function.identity(), - null - ); - } - - @Override - public int getPriority() - { - return priority; - } - - public void markReady() - { - readyLatch.countDown(); - } - - public void beginPublish() - { - runLatch.countDown(); - } - - public void awaitReadyComplete() throws InterruptedException - { - readyComplete.await(); - } - - public void completeSegmentAllocation() - { - segmentAllocationComplete.countDown(); - } - - public void awaitRunComplete() throws InterruptedException - { - runComplete.await(); - } - -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java deleted file mode 100644 index 2b8ee4b0a863..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ReplaceTask.java +++ /dev/null @@ -1,253 +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.batch.parallel; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -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.TaskToolbox; -import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; -import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; -import org.apache.druid.indexing.common.actions.SurrogateTaskActionClient; -import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; -import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.task.AbstractTask; -import org.apache.druid.indexing.overlord.SegmentPublishResult; -import org.apache.druid.indexing.overlord.Segments; -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.granularity.Granularity; -import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; -import org.apache.druid.server.security.ResourceAction; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.apache.druid.timeline.partition.ShardSpec; -import org.joda.time.Interval; - -import javax.annotation.Nonnull; -import java.io.IOException; -import java.util.HashSet; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.CountDownLatch; -import java.util.function.Function; - -/** - * A test Task which mimics a replacing task by having similar interactions with the overlord. - * - * Begins running by acquiring a REPLACE lock - * - * Task ends after publishing a set of core partitions and - * creating metadata copies for all appended segments published when this lock was held - */ -public class ReplaceTask extends AbstractTask -{ - private final Interval interval; - private final Granularity segmentGranularity; - private final TaskLockType lockType; - private final int priority; - private final int numCorePartitions; - private final CountDownLatch readyLatch = new CountDownLatch(1); - private final CountDownLatch readyComplete = new CountDownLatch(1); - private final CountDownLatch runLatch = new CountDownLatch(1); - private final CountDownLatch runComplete = new CountDownLatch(1); - private String version; - - public ReplaceTask( - String id, - String dataSource, - Interval interval, - Granularity segmentGranularity, - Integer priority, - Integer numCorePartitions - ) - { - super( - id == null ? StringUtils.format("replace_%s_%s", DateTimes.nowUtc(), UUID.randomUUID().toString()) : id, - dataSource == null ? "none" : dataSource, - null, - IngestionMode.REPLACE - ); - this.interval = interval; - this.segmentGranularity = segmentGranularity; - this.lockType = TaskLockType.REPLACE; - this.priority = priority == null ? 50 : priority; - this.numCorePartitions = numCorePartitions == null ? 1 : numCorePartitions; - } - - @Override - public String getType() - { - return "replace"; - } - - @Nonnull - @Override - public Set getInputSourceResources() - { - return ImmutableSet.of(); - } - - @Override - public String setup(TaskToolbox toolbox) throws Exception - { - readyLatch.await(); - while (!isReady(toolbox.getTaskActionClient())) { - Thread.sleep(100); - } - return null; - } - - @Override - public boolean isReady(TaskActionClient taskActionClient) throws Exception - { - return tryTimeChunkLockSingleInterval( - new SurrogateTaskActionClient(getId(), taskActionClient), - interval, - lockType - ); - } - - private boolean tryTimeChunkLockSingleInterval(TaskActionClient client, Interval interval, TaskLockType lockType) - throws IOException - { - final TaskLock lock = client.submit(new TimeChunkLockTryAcquireAction(lockType, interval)); - if (lock == null) { - return false; - } - if (lock.isRevoked()) { - throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", interval)); - } - version = lock.getVersion(); - return true; - } - - private Set findSegmentsToLock(TaskActionClient taskActionClient) - throws IOException - { - return ImmutableSet.copyOf( - taskActionClient.submit( - new RetrieveUsedSegmentsAction(getDataSource(), null, ImmutableList.of(interval), Segments.ONLY_VISIBLE) - ) - ); - } - - @Override - public void stopGracefully(TaskConfig taskConfig) - { - } - - @Override - public TaskStatus runTask(TaskToolbox toolbox) throws Exception - { - final Set oldSegments = findSegmentsToLock(toolbox.getTaskActionClient()); - - readyComplete.countDown(); - - runLatch.await(); - - final Set newSegments = createSegments(); - final SegmentPublishResult publishResult = publishSegments(toolbox, oldSegments, newSegments); - if (publishResult.isSuccess()) { - return TaskStatus.success(getId()); - } else { - return TaskStatus.failure(getId(), publishResult.getErrorMsg()); - } - } - - @Override - public void cleanUp(TaskToolbox toolbox, TaskStatus taskStatus) throws Exception - { - super.cleanUp(toolbox, taskStatus); - runComplete.countDown(); - } - - - private Set createSegments() - { - final Set newSegments = new HashSet<>(); - for (int i = 0; i < numCorePartitions; i++) { - for (Interval subInterval : segmentGranularity.getIterable(interval)) { - final ShardSpec shardSpec = new NumberedShardSpec(i, numCorePartitions); - final SegmentId segmentId = SegmentId.of(getDataSource(), subInterval, version, shardSpec); - newSegments.add( - new DataSegment( - segmentId, - ImmutableMap.of(segmentId.toString(), segmentId.toString()), - ImmutableList.of(), - ImmutableList.of(), - shardSpec, - null, - 0, - 0 - ) - ); - } - } - return newSegments; - } - - private SegmentPublishResult publishSegments(TaskToolbox toolbox, Set oldSegments, Set newSegments) - throws Exception - { - final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> - toolbox.getTaskActionClient().submit( - SegmentTransactionalReplaceAction.create(segmentsToPublish) - ); - return publisher.publishSegments( - oldSegments, - newSegments, - Function.identity(), - null - ); - } - - @Override - public int getPriority() - { - return priority; - } - - public void awaitRunComplete() throws InterruptedException - { - runComplete.await(); - } - - public void awaitReadyComplete() throws InterruptedException - { - readyComplete.await(); - } - - public void markReady() - { - readyLatch.countDown(); - } - - public void beginPublish() - { - runLatch.countDown(); - } -} diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index c88f114a7e52..b2d4ab22a0e6 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1085,27 +1085,32 @@ Map> getSegmentsToUpgradeOnAppend( .add(segment); } - // Fetch all used non-overshadowed segments that overlap with any of the append intervals - final Collection overlappingSegments - = retrieveUsedSegmentsForIntervals(dataSource, new ArrayList<>(appendIntervals), Segments.ONLY_VISIBLE); + // Fetch all used segments that overlap with any of the append intervals + final Collection overlappingSegments = retrieveUsedSegmentsForIntervals( + dataSource, + new ArrayList<>(appendIntervals), + Segments.INCLUDING_OVERSHADOWED + ); final Set committedVersions = new HashSet<>(); final Map> committedIntervalToSegments = new HashMap<>(); for (DataSegment segment : overlappingSegments) { committedVersions.add(segment.getVersion()); committedIntervalToSegments.computeIfAbsent(segment.getInterval(), i -> new HashSet<>()) - .add(segment); + .add(segment); } final Map> appendSegmentToNewIds = new HashMap<>(); - for (String version : committedVersions) { - Map> committedIntervalToCarrySegments - = getIntervalToCarrySegments(version, committedIntervalToSegments.keySet(), appendVersionToSegments); - for (Map.Entry> entry : committedIntervalToCarrySegments.entrySet()) { + for (String upgradeVersion : committedVersions) { + Map> segmentsToUpgrade = getSegmentsWithVersionLowerThan( + upgradeVersion, + committedIntervalToSegments.keySet(), + appendVersionToSegments + ); + for (Map.Entry> entry : segmentsToUpgrade.entrySet()) { computeNewAppendIdsForVersion( handle, - version, - dataSource, + upgradeVersion, entry.getKey(), entry.getValue(), committedIntervalToSegments, @@ -1117,55 +1122,60 @@ Map> getSegmentsToUpgradeOnAppend( return appendSegmentToNewIds; } - private Map> getIntervalToCarrySegments( - String version, - Set committedIntervals, - TreeMap> appendVersionToSegments + /** + * Creates a Map from eligible interval to Set of segments that are fully + * contained in that interval and have a version strictly lower than {@code #cutoffVersion}. + */ + private Map> getSegmentsWithVersionLowerThan( + String cutoffVersion, + Set eligibleIntervals, + TreeMap> versionToSegments ) { - // Find all the append segments with version strictly less than this version - final Set segmentsToCarryForward = new HashSet<>(); - - Map> candidateVersionToSegments = appendVersionToSegments.headMap(version); - for (Set segments : candidateVersionToSegments.values()) { - segmentsToCarryForward.addAll(segments); - } - - final Map> committedIntervalToAppendSegments = new HashMap<>(); - for (DataSegment appendSegment : segmentsToCarryForward) { - final Interval appendSegmentInterval = appendSegment.getInterval(); - for (Interval committedInterval : committedIntervals) { - if (committedInterval.contains(appendSegmentInterval)) { - committedIntervalToAppendSegments.computeIfAbsent(committedInterval, itvl -> new HashSet<>()) - .add(appendSegment); + final Set eligibleSegments + = versionToSegments.headMap(cutoffVersion).values().stream() + .flatMap(Collection::stream) + .collect(Collectors.toSet()); + + final Map> eligibleIntervalToSegments = new HashMap<>(); + + for (DataSegment segment : eligibleSegments) { + final Interval segmentInterval = segment.getInterval(); + for (Interval eligibleInterval : eligibleIntervals) { + if (eligibleInterval.contains(segmentInterval)) { + eligibleIntervalToSegments.computeIfAbsent(eligibleInterval, itvl -> new HashSet<>()) + .add(segment); break; - } else if (committedInterval.overlaps(appendSegmentInterval)) { + } else if (eligibleInterval.overlaps(segmentInterval)) { // Committed interval overlaps only partially throw new ISE( "Committed interval[%s] conflicts with interval[%s] of append segment[%s].", - committedInterval, appendSegmentInterval, appendSegment.getId() + eligibleInterval, segmentInterval, segment.getId() ); } } } - return committedIntervalToAppendSegments; + return eligibleIntervalToSegments; } + /** + * Computes new Segment IDs for the {@code segmentsToUpgrade} being upgraded + * to the given {@code upgradeVersion}. + */ private void computeNewAppendIdsForVersion( Handle handle, - String committedVersion, - String dataSource, + String upgradeVersion, Interval interval, - Set carrySegments, - Map> committedIntervalToSegments, + Set segmentsToUpgrade, + Map> committedSegmentsByInterval, Map> appendSegmentToNewIds ) throws IOException { final Set committedSegments - = committedIntervalToSegments.getOrDefault(interval, Collections.emptySet()) + = committedSegmentsByInterval.getOrDefault(interval, Collections.emptySet()) .stream() - .filter(s -> s.getVersion().equals(committedVersion)) + .filter(s -> s.getVersion().equals(upgradeVersion)) .collect(Collectors.toSet()); SegmentIdWithShardSpec committedMaxId = null; @@ -1177,16 +1187,17 @@ private void computeNewAppendIdsForVersion( } // Get pending segments for the new version, if any + final String dataSource = segmentsToUpgrade.iterator().next().getDataSource(); final Set pendingSegments = getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval); // Determine new IDs for each append segment by taking into account both // committed and pending segments for this version - for (DataSegment segment : carrySegments) { + for (DataSegment segment : segmentsToUpgrade) { SegmentCreateRequest request = new SegmentCreateRequest( - segment.getId() + committedVersion, + segment.getId() + "__" + upgradeVersion, null, - committedVersion, + upgradeVersion, NumberedPartialShardSpec.instance() ); // allocate new segment id @@ -1194,7 +1205,7 @@ private void computeNewAppendIdsForVersion( request, dataSource, interval, - committedVersion, + upgradeVersion, committedMaxId, pendingSegments ); diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 2794381d6f10..bf59b2228841 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -632,28 +632,26 @@ public void testAllocateNewSegmentIds() for (DataSegment segment : day1) { final Set newIds = segmentToNewIds.get(segment); - // TODO: this test originally expected 2 new IDs but older version would already be overshadowed - Assert.assertEquals(1, newIds.size()); + Assert.assertEquals(2, newIds.size()); Assert.assertEquals( - ImmutableSet.of(v2), + ImmutableSet.of(v1, v2), newIds.stream().map(SegmentIdWithShardSpec::getVersion).collect(Collectors.toSet()) ); } for (DataSegment segment : day2) { final Set newIds = segmentToNewIds.get(segment); - // TODO: this test originally expected 2 new IDs but older version would already be overshadowed - Assert.assertEquals(1, newIds.size()); + Assert.assertEquals(2, newIds.size()); Assert.assertEquals( - ImmutableSet.of(v2), + ImmutableSet.of(v1, v2), newIds.stream().map(SegmentIdWithShardSpec::getVersion).collect(Collectors.toSet()) ); } for (DataSegment segment : day3) { final Set newIds = segmentToNewIds.get(segment); - Assert.assertEquals(1, newIds.size()); + Assert.assertEquals(2, newIds.size()); Assert.assertEquals( - ImmutableSet.of(v2), + ImmutableSet.of(v1, v2), newIds.stream().map(SegmentIdWithShardSpec::getVersion).collect(Collectors.toSet()) ); } From ff7167499eb32bfe540dd30284d1b372f3cfcb37 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 6 Sep 2023 10:51:41 +0530 Subject: [PATCH 28/43] Fix synchronization in CommandQueueTask --- .../common/task/CommandQueueTask.java | 74 ++++++++++++------- .../task/ConcurrentReplaceAndAppendTest.java | 23 +++--- .../task/batch/parallel/ActionsTestTask.java | 26 ++++++- 3 files changed, 84 insertions(+), 39 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandQueueTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandQueueTask.java index 8f0aefc49d76..beaf7c73c633 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandQueueTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandQueueTask.java @@ -4,7 +4,6 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; -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.logger.Logger; @@ -17,7 +16,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; /** * Test task that can be given a series of commands to execute in its {@link #runTask} method. @@ -26,16 +24,17 @@ public class CommandQueueTask extends AbstractTask { private static final Logger log = new Logger(CommandQueueTask.class); + private final Object queueNotification = new Object(); private final BlockingQueue> commandQueue = new LinkedBlockingQueue<>(); private final AtomicBoolean finishRequested = new AtomicBoolean(false); private final AtomicInteger numCommandsExecuted = new AtomicInteger(0); - private final AtomicReference finalTaskStatus = new AtomicReference<>(); + private final CompletableFuture finalTaskStatus = new CompletableFuture<>(); public CommandQueueTask(String datasource, String groupId) { super( - StringUtils.format("test_%s_%s", DateTimes.nowUtc(), UUID.randomUUID().toString()), + StringUtils.format("test_%s_%s", datasource, UUID.randomUUID().toString()), groupId, null, datasource, @@ -50,13 +49,19 @@ public CommandQueueTask(String datasource, String groupId) */ public TaskStatus finishRunAndGetStatus() { - // Mark finished to prevent submission of any more commands - finishRequested.set(true); - - // Submit a dummy command to ensure that all previous commands have finished - executeInternal(() -> 1); + synchronized (finishRequested) { + finishRequested.set(true); + } + synchronized (queueNotification) { + queueNotification.notify(); + } - return finalTaskStatus.get(); + try { + return finalTaskStatus.get(10, TimeUnit.SECONDS); + } + catch (Exception e) { + throw new ISE(e, "Error waiting for task[%s] to finish", getId()); + } } /** @@ -65,10 +70,6 @@ public TaskStatus finishRunAndGetStatus() */ public void submit(Runnable runnable) { - if (finishRequested.get()) { - throw new ISE("Task[%s] cannot accept any more commands as it is already shutting down.", getId()); - } - // Add a command with a dummy return value Command command = new Command<>( () -> { @@ -76,7 +77,7 @@ public void submit(Runnable runnable) return 1; } ); - commandQueue.offer(command); + addToQueue(command); } /** @@ -85,18 +86,28 @@ public void submit(Runnable runnable) */ public V execute(Callable callable) { - if (finishRequested.get()) { - throw new ISE("Task[%s] cannot accept any more commands as it is already shutting down.", getId()); + Command command = new Command<>(callable); + addToQueue(command); + return waitForCommandToFinish(command); + } + + private void addToQueue(Command command) + { + synchronized (finishRequested) { + if (finishRequested.get()) { + throw new ISE("Task[%s] cannot accept any more commands as it is already shutting down.", getId()); + } else { + commandQueue.offer(command); + } } - return executeInternal(callable); + synchronized (queueNotification) { + queueNotification.notify(); + } } - private V executeInternal(Callable callable) + private V waitForCommandToFinish(Command command) { - Command command = new Command<>(callable); - commandQueue.offer(command); - try { return command.value.get(10, TimeUnit.SECONDS); } @@ -110,9 +121,20 @@ public TaskStatus runTask(TaskToolbox taskToolbox) throws Exception { TaskStatus status; try { - while (commandQueue.size() > 0 || !finishRequested.get()) { - Command command = commandQueue.poll(10, TimeUnit.SECONDS); - if (command != null) { + while (true) { + synchronized (finishRequested) { + if (finishRequested.get() && commandQueue.isEmpty()) { + break; + } + } + + Command command = commandQueue.poll(); + if (command == null) { + synchronized (queueNotification) { + queueNotification.wait(10_000); + } + } else { + log.info("Running command[%d] for task[%s]", numCommandsExecuted.get(), getId()); command.execute(); numCommandsExecuted.incrementAndGet(); } @@ -124,7 +146,7 @@ public TaskStatus runTask(TaskToolbox taskToolbox) throws Exception status = TaskStatus.failure(getId(), e.getMessage()); } - finalTaskStatus.set(status); + finalTaskStatus.complete(status); return status; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java index 96c7467a4ec0..f7cd3e451cf2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java @@ -41,6 +41,7 @@ import org.apache.druid.indexing.overlord.config.TaskQueueConfig; import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; @@ -119,14 +120,14 @@ public void testAppendSegmentGetsUpgraded() throws Exception final Interval year2023 = Intervals.of("2023/2024"); // Commit initial segments for v0 - ActionsTestTask replaceTask0 = createAndStartTask(); - TaskLock replaceLock = replaceTask0.acquireReplaceLockOn(year2023); + final ActionsTestTask replaceTask0 = createAndStartTask(); + final String v0 = replaceTask0.acquireReplaceLockOn(year2023).getVersion(); final DataSegment segmentV00 = DataSegment.builder() .dataSource(DS.WIKI) .interval(year2023) - .version(replaceLock.getVersion()) + .version(v0) .size(1) .build(); @@ -138,28 +139,28 @@ public void testAppendSegmentGetsUpgraded() throws Exception // Allocate an append segment for v0 final ActionsTestTask appendTask0 = createAndStartTask(); appendTask0.acquireAppendLockOn(year2023); - // TODO: fix allocation and version of allocated segment - final SegmentIdWithShardSpec pendingSegmentV01 = appendTask0.allocateSegment(); + final SegmentIdWithShardSpec pendingSegmentV01 + = appendTask0.allocateSegmentForTimestamp(year2023.getStart(), Granularities.YEAR); // Commit replace segment for v1 final ActionsTestTask replaceTask1 = createAndStartTask(); - replaceTask1.acquireReplaceLockOn(year2023); + final String v1 = replaceTask1.acquireReplaceLockOn(year2023).getVersion(); - final DataSegment segmentV10 = DataSegment.builder(segmentV00).version("v1").build(); + final DataSegment segmentV10 = DataSegment.builder(segmentV00).version(v1).build(); replaceTask1.commitReplaceSegments(segmentV10); replaceTask1.finishRunAndGetStatus(); verifyIntervalHasUsedSegments(year2023, segmentV00, segmentV10); verifyIntervalHasVisibleSegments(year2023, segmentV10); final ActionsTestTask replaceTask2 = createAndStartTask(); - replaceTask2.acquireReplaceLockOn(year2023); + final String v2 = replaceTask2.acquireReplaceLockOn(year2023).getVersion(); // Commit append segment v0 and verify that it gets upgraded to v1 final DataSegment segmentV01 = asSegment(pendingSegmentV01); appendTask0.commitAppendSegments(segmentV01); appendTask0.finishRunAndGetStatus(); - final DataSegment segmentV11 = DataSegment.builder(segmentV01).version(segmentV10.getVersion()).build(); + final DataSegment segmentV11 = DataSegment.builder(segmentV01).version(v1).build(); verifyIntervalHasUsedSegments( year2023, segmentV00, segmentV01, segmentV10, segmentV11 @@ -167,11 +168,11 @@ public void testAppendSegmentGetsUpgraded() throws Exception verifyIntervalHasVisibleSegments(year2023, segmentV10, segmentV11); // Commit replace segment v2 and verify that append segment gets upgraded to v2 - final DataSegment segmentV20 = DataSegment.builder(segmentV00).version("v2").build(); + final DataSegment segmentV20 = DataSegment.builder(segmentV00).version(v2).build(); replaceTask2.commitReplaceSegments(segmentV20); replaceTask2.finishRunAndGetStatus(); - final DataSegment segmentV21 = DataSegment.builder(segmentV01).version(segmentV20.getVersion()).build(); + final DataSegment segmentV21 = DataSegment.builder(segmentV01).version(v2).build(); verifyIntervalHasUsedSegments( year2023, segmentV00, segmentV01, segmentV10, segmentV11, segmentV20, segmentV21 diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ActionsTestTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ActionsTestTask.java index 07e67a87560b..035d791238d3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ActionsTestTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ActionsTestTask.java @@ -1,8 +1,10 @@ package org.apache.druid.indexing.common.task.batch.parallel; import com.google.common.collect.Sets; +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.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; import org.apache.druid.indexing.common.actions.TaskAction; @@ -12,16 +14,23 @@ import org.apache.druid.indexing.common.task.CommandQueueTask; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NumberedPartialShardSpec; +import org.joda.time.DateTime; import org.joda.time.Interval; +import java.util.concurrent.atomic.AtomicInteger; + /** * Test task that can only invoke task actions. */ public class ActionsTestTask extends CommandQueueTask { private final TaskActionClient client; + private final AtomicInteger sequenceId = new AtomicInteger(0); public ActionsTestTask(String datasource, TaskActionClientFactory factory) { @@ -53,9 +62,22 @@ public SegmentPublishResult commitAppendSegments(DataSegment... segments) ); } - public SegmentIdWithShardSpec allocateSegment() + public SegmentIdWithShardSpec allocateSegmentForTimestamp(DateTime timestamp, Granularity preferredSegmentGranularity) { - return null; + return runAction( + new SegmentAllocateAction( + getDataSource(), + timestamp, + Granularities.SECOND, + preferredSegmentGranularity, + getId() + "__" + sequenceId.getAndIncrement(), + null, + false, + NumberedPartialShardSpec.instance(), + LockGranularity.TIME_CHUNK, + TaskLockType.APPEND + ) + ); } private TaskLock tryTimeChunkLock(Interval interval, TaskLockType lockType) From 4e0587f0e0e614860c3314dc9f6feece96431c9a Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 7 Sep 2023 19:51:49 +0530 Subject: [PATCH 29/43] Move some tests --- .../common/task/CommandQueueTask.java | 6 +- .../task/ConcurrentReplaceAndAppendTest.java | 190 ++++++++---------- 2 files changed, 90 insertions(+), 106 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandQueueTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandQueueTask.java index beaf7c73c633..46d8060fa557 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandQueueTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandQueueTask.java @@ -26,6 +26,7 @@ public class CommandQueueTask extends AbstractTask private final Object queueNotification = new Object(); private final BlockingQueue> commandQueue = new LinkedBlockingQueue<>(); + private final AtomicBoolean finishRequested = new AtomicBoolean(false); private final AtomicInteger numCommandsExecuted = new AtomicInteger(0); @@ -97,7 +98,10 @@ private void addToQueue(Command command) if (finishRequested.get()) { throw new ISE("Task[%s] cannot accept any more commands as it is already shutting down.", getId()); } else { - commandQueue.offer(command); + boolean added = commandQueue.offer(command); + if (!added) { + throw new ISE("Could not add command to task[%s].", getId()); + } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java index f7cd3e451cf2..cd468e0f3706 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java @@ -22,7 +22,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import org.apache.druid.indexing.common.MultipleFileTaskReportFileWriter; -import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskStorageDirTracker; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolboxFactory; @@ -35,11 +34,13 @@ import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.overlord.TaskQueue; import org.apache.druid.indexing.overlord.TaskRunner; +import org.apache.druid.indexing.overlord.TestTaskToolboxFactory; import org.apache.druid.indexing.overlord.ThreadingTaskRunner; import org.apache.druid.indexing.overlord.config.DefaultTaskConfig; import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.indexing.overlord.config.TaskQueueConfig; import org.apache.druid.indexing.worker.config.WorkerConfig; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.IndexIO; @@ -57,10 +58,12 @@ import org.junit.Before; import org.junit.Test; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Set; public class ConcurrentReplaceAndAppendTest extends IngestionTestBase { @@ -78,9 +81,8 @@ public void setup() final TaskConfig taskConfig = new TaskConfigBuilder().build(); taskActionClientFactory = createActionClientFactory(); dummyTaskActionClient = taskActionClientFactory.create(NoopTask.create()); - final TaskToolboxFactory toolboxFactory = new TestTaskToolboxFactory(taskConfig, taskActionClientFactory); taskRunner = new ThreadingTaskRunner( - toolboxFactory, + createToolboxFactory(taskConfig, taskActionClientFactory), taskConfig, WORKER_CONFIG, new NoopTaskLogs(), @@ -115,7 +117,7 @@ public void tearDown() } @Test - public void testAppendSegmentGetsUpgraded() throws Exception + public void testAppendSegmentGetsUpgraded() { final Interval year2023 = Intervals.of("2023/2024"); @@ -123,14 +125,7 @@ public void testAppendSegmentGetsUpgraded() throws Exception final ActionsTestTask replaceTask0 = createAndStartTask(); final String v0 = replaceTask0.acquireReplaceLockOn(year2023).getVersion(); - final DataSegment segmentV00 - = DataSegment.builder() - .dataSource(DS.WIKI) - .interval(year2023) - .version(v0) - .size(1) - .build(); - + final DataSegment segmentV00 = createSegment(year2023, v0); replaceTask0.commitReplaceSegments(segmentV00); replaceTask0.finishRunAndGetStatus(); verifyIntervalHasUsedSegments(year2023, segmentV00); @@ -141,6 +136,7 @@ public void testAppendSegmentGetsUpgraded() throws Exception appendTask0.acquireAppendLockOn(year2023); final SegmentIdWithShardSpec pendingSegmentV01 = appendTask0.allocateSegmentForTimestamp(year2023.getStart(), Granularities.YEAR); + Assert.assertEquals(segmentV00.getVersion(), pendingSegmentV01.getVersion()); // Commit replace segment for v1 final ActionsTestTask replaceTask1 = createAndStartTask(); @@ -157,10 +153,11 @@ public void testAppendSegmentGetsUpgraded() throws Exception // Commit append segment v0 and verify that it gets upgraded to v1 final DataSegment segmentV01 = asSegment(pendingSegmentV01); - appendTask0.commitAppendSegments(segmentV01); - appendTask0.finishRunAndGetStatus(); - final DataSegment segmentV11 = DataSegment.builder(segmentV01).version(v1).build(); + Set appendedSegments = appendTask0.commitAppendSegments(segmentV01).getSegments(); + Assert.assertEquals(Sets.newHashSet(segmentV01, segmentV11), appendedSegments); + + appendTask0.finishRunAndGetStatus(); verifyIntervalHasUsedSegments( year2023, segmentV00, segmentV01, segmentV10, segmentV11 @@ -180,6 +177,34 @@ public void testAppendSegmentGetsUpgraded() throws Exception verifyIntervalHasVisibleSegments(year2023, segmentV20, segmentV21); } + @Test + public void testRRAA_dailyReplaceDailyAppend() + { + final Interval firstOfJan = Intervals.of("2023-01-01/2023-01-02"); + + final ActionsTestTask replaceTask0 = createAndStartTask(); + final ActionsTestTask appendTask0 = createAndStartTask(); + + final String v0 = replaceTask0.acquireReplaceLockOn(firstOfJan).getVersion(); + + final DataSegment segmentV00 = createSegment(firstOfJan, v0); + + replaceTask0.commitReplaceSegments(segmentV00); + replaceTask0.finishRunAndGetStatus(); + verifyIntervalHasUsedSegments(firstOfJan, segmentV00); + + appendTask0.acquireAppendLockOn(firstOfJan); + final SegmentIdWithShardSpec pendingSegment + = appendTask0.allocateSegmentForTimestamp(firstOfJan.getStart(), Granularities.DAY); + Assert.assertEquals(segmentV00.getVersion(), pendingSegment.getVersion()); + + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask0.commitAppendSegments(segmentV01); + + verifyIntervalHasUsedSegments(firstOfJan, segmentV00, segmentV01); + verifyIntervalHasVisibleSegments(firstOfJan, segmentV00, segmentV01); + } + private static DataSegment asSegment(SegmentIdWithShardSpec pendingSegment) { final SegmentId id = pendingSegment.asSegmentId(); @@ -195,119 +220,74 @@ private static DataSegment asSegment(SegmentIdWithShardSpec pendingSegment) ); } - private void verifyTaskSuccess(Task task) - { - try { - while (!getTaskStorage().getStatus(task.getId()).get().isComplete()) { - Thread.sleep(100); - } - } - catch (InterruptedException e) { - // do nothing - } - Assert.assertTrue(getTaskStorage().getStatus(task.getId()).get().isSuccess()); - } - - private void verifyTaskFailure(Task task) - { - try { - while (!getTaskStorage().getStatus(task.getId()).get().isComplete()) { - Thread.sleep(100); - } - } - catch (InterruptedException e) { - // do nothing - } - Assert.assertTrue(getTaskStorage().getStatus(task.getId()).get().isFailure()); - } - - private void verifyIntervalHasUsedSegments(Interval interval, DataSegment... expectedSegments) throws Exception + private void verifyIntervalHasUsedSegments(Interval interval, DataSegment... expectedSegments) { verifySegments(interval, Segments.INCLUDING_OVERSHADOWED, expectedSegments); } - private void verifyIntervalHasVisibleSegments(Interval interval, DataSegment... expectedSegments) throws Exception + private void verifyIntervalHasVisibleSegments(Interval interval, DataSegment... expectedSegments) { verifySegments(interval, Segments.ONLY_VISIBLE, expectedSegments); } - private void verifySegments(Interval interval, Segments visibility, DataSegment... expectedSegments) throws Exception + private void verifySegments(Interval interval, Segments visibility, DataSegment... expectedSegments) { - Collection allUsedSegments = dummyTaskActionClient.submit( - new RetrieveUsedSegmentsAction( - DS.WIKI, - null, - ImmutableList.of(interval), - visibility - ) - ); - Assert.assertEquals(Sets.newHashSet(expectedSegments), Sets.newHashSet(allUsedSegments)); - } - - private class TestTaskToolboxFactory extends TaskToolboxFactory - { - public TestTaskToolboxFactory(TaskConfig taskConfig, TaskActionClientFactory taskActionClientFactory) - { - super( - taskConfig, - null, - taskActionClientFactory, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - new IndexIO(getObjectMapper(), ColumnConfig.DEFAULT), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null + try { + Collection allUsedSegments = dummyTaskActionClient.submit( + new RetrieveUsedSegmentsAction( + DS.WIKI, + null, + ImmutableList.of(interval), + visibility + ) ); + Assert.assertEquals(Sets.newHashSet(expectedSegments), Sets.newHashSet(allUsedSegments)); + } + catch (IOException e) { + throw new ISE(e, "Error while fetching used segments in interval[%s]", interval); } + } - @Override - public TaskToolbox build(TaskConfig config, Task task) + private TaskToolboxFactory createToolboxFactory( + TaskConfig taskConfig, + TaskActionClientFactory taskActionClientFactory + ) + { + TestTaskToolboxFactory.Builder builder = new TestTaskToolboxFactory.Builder() + .setConfig(taskConfig) + .setIndexIO(new IndexIO(getObjectMapper(), ColumnConfig.DEFAULT)) + .setTaskActionClientFactory(taskActionClientFactory); + return new TestTaskToolboxFactory(builder) { - return createTaskToolbox(config, task); - } + @Override + public TaskToolbox build(TaskConfig config, Task task) + { + return createTaskToolbox(config, task); + } + }; } - private static class DS + private DataSegment createSegment(Interval interval, String version) { - static final String WIKI = "wiki"; + return DataSegment.builder() + .dataSource(DS.WIKI) + .interval(interval) + .version(version) + .size(100) + .build(); } private ActionsTestTask createAndStartTask() { - ActionsTestTask task = new ActionsTestTask("wiki", taskActionClientFactory); + ActionsTestTask task = new ActionsTestTask(DS.WIKI, taskActionClientFactory); taskQueue.add(task); runningTasks.add(task); return task; } + private static class DS + { + static final String WIKI = "wiki"; + } + } From 359a923c94c0e91cec9d4af2defb25c53044163f Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 11 Sep 2023 08:26:34 +0530 Subject: [PATCH 30/43] Add more tests --- concurrent.diff | 1093 +++++++++++++++++ .../task/ConcurrentReplaceAndAppendTest.java | 293 ----- .../ActionsTestTask.java | 22 +- .../{ => concurrent}/CommandQueueTask.java | 30 +- .../ConcurrentReplaceAppendTest.java | 447 +++++++ 5 files changed, 1585 insertions(+), 300 deletions(-) create mode 100644 concurrent.diff delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java rename indexing-service/src/test/java/org/apache/druid/indexing/common/task/{batch/parallel => concurrent}/ActionsTestTask.java (79%) rename indexing-service/src/test/java/org/apache/druid/indexing/common/task/{ => concurrent}/CommandQueueTask.java (82%) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAppendTest.java diff --git a/concurrent.diff b/concurrent.diff new file mode 100644 index 000000000000..0b0796dc8b4f --- /dev/null +++ b/concurrent.diff @@ -0,0 +1,1093 @@ +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +index e0d155408d..95f3a96aca 100644 +--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +@@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; + import com.fasterxml.jackson.annotation.JsonProperty; + import com.fasterxml.jackson.core.type.TypeReference; + import com.google.common.collect.ImmutableSet; +-import org.apache.druid.indexing.common.TaskLock; + import org.apache.druid.indexing.common.task.IndexTaskUtils; + import org.apache.druid.indexing.common.task.Task; + import org.apache.druid.indexing.overlord.CriticalAction; +@@ -31,14 +30,10 @@ import org.apache.druid.indexing.overlord.DataSourceMetadata; + import org.apache.druid.indexing.overlord.SegmentPublishResult; + import org.apache.druid.indexing.overlord.TaskLockInfo; + import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +-import org.apache.druid.query.DruidMetrics; + import org.apache.druid.segment.SegmentUtils; + import org.apache.druid.timeline.DataSegment; +-import org.joda.time.Interval; + + import javax.annotation.Nullable; +-import java.util.HashMap; +-import java.util.HashSet; + import java.util.Map; + import java.util.Set; + import java.util.stream.Collectors; +@@ -111,42 +106,22 @@ public class SegmentTransactionalAppendAction implements TaskAction allSegments = new HashSet<>(segments); +- +- String datasource = task.getDataSource(); +- Map replaceLocks = new HashMap<>(); +- for (TaskLock lock : TaskLocks.findReplaceLocksForSegments(datasource, toolbox.getTaskLockbox(), segments)) { +- replaceLocks.put(lock.getInterval(), lock); +- } +- Map appendSegmentLockMap = new HashMap<>(); +- Set taskLockInfos = new HashSet<>(); +- for (TaskLock taskLock : replaceLocks.values()) { +- taskLockInfos.add(getTaskLockInfo(taskLock)); +- } +- +- for (DataSegment segment : segments) { +- Interval interval = segment.getInterval(); +- for (Interval key : replaceLocks.keySet()) { +- if (key.contains(interval)) { +- appendSegmentLockMap.put(segment, getTaskLockInfo(replaceLocks.get(key))); +- } +- } +- } ++ final String datasource = task.getDataSource(); ++ final Map segmentToReplaceLock ++ = TaskLocks.findReplaceLocksCoveringSegments(datasource, toolbox.getTaskLockbox(), segments); + ++ final SegmentPublishResult retVal; + try { + retVal = toolbox.getTaskLockbox().doInCriticalSection( + task, +- allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), ++ segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), + CriticalAction.builder() + .onValidLocks( + () -> toolbox.getIndexerMetadataStorageCoordinator().commitAppendSegments( + segments, + startMetadata, + endMetadata, +- appendSegmentLockMap, +- taskLockInfos ++ segmentToReplaceLock + ) + ) + .onInvalidLocks( +@@ -168,29 +143,17 @@ public class SegmentTransactionalAppendAction implements TaskActionbuilder() + .onValidLocks( +- () -> toolbox.getIndexerMetadataStorageCoordinator().commitSegments( ++ () -> toolbox.getIndexerMetadataStorageCoordinator().commitSegmentsAndMetadata( + segments, + segmentsToBeDropped, + startMetadata, +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +index 4dd3ea7a38..255628e262 100644 +--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +@@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; + import com.fasterxml.jackson.annotation.JsonProperty; + import com.fasterxml.jackson.core.type.TypeReference; + import com.google.common.collect.ImmutableSet; +-import org.apache.druid.indexing.common.TaskLock; + import org.apache.druid.indexing.common.task.IndexTaskUtils; + import org.apache.druid.indexing.common.task.Task; + import org.apache.druid.indexing.overlord.CriticalAction; +@@ -33,10 +32,8 @@ import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; + import org.apache.druid.query.DruidMetrics; + import org.apache.druid.segment.SegmentUtils; + import org.apache.druid.timeline.DataSegment; +-import org.joda.time.Interval; + + import javax.annotation.Nullable; +-import java.util.HashMap; + import java.util.HashSet; + import java.util.Map; + import java.util.Set; +@@ -119,32 +116,21 @@ public class SegmentTransactionalReplaceAction implements TaskAction allSegments = new HashSet<>(segments); +- +- TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), allSegments); +- + String datasource = task.getDataSource(); +- Map replaceLocks = new HashMap<>(); +- for (TaskLock lock : TaskLocks.findReplaceLocksForSegments(datasource, toolbox.getTaskLockbox(), segments)) { +- replaceLocks.put(lock.getInterval(), lock); +- } +- Set taskLockInfos = new HashSet<>(); +- for (TaskLock taskLock : replaceLocks.values()) { +- taskLockInfos.add(getTaskLockInfo(taskLock)); +- } ++ final Map segmentToReplaceLock ++ = TaskLocks.findReplaceLocksCoveringSegments(datasource, toolbox.getTaskLockbox(), segments); + ++ final SegmentPublishResult retVal; + try { + retVal = toolbox.getTaskLockbox().doInCriticalSection( + task, +- allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), ++ segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), + CriticalAction.builder() + .onValidLocks( + () -> toolbox.getIndexerMetadataStorageCoordinator().commitReplaceSegments( + segments, + segmentsToBeDropped, +- taskLockInfos ++ new HashSet<>(segmentToReplaceLock.values()) + ) + ) + .onInvalidLocks( +@@ -166,28 +152,20 @@ public class SegmentTransactionalReplaceAction implements TaskAction findReplaceLocksForSegments( ++ /** ++ * Finds locks of type {@link TaskLockType#REPLACE} for each of the given segments ++ * that have an interval completely covering the interval of the respective segments. ++ * ++ * @return Map from segment to REPLACE lock that completely covers it. The map ++ * does not contain an entry for segments that have no covering REPLACE lock. ++ */ ++ public static Map findReplaceLocksCoveringSegments( + final String datasource, + final TaskLockbox taskLockbox, +- final Collection segments ++ final Set segments + ) + { +- final Set found = new HashSet<>(); +- final Set locks = taskLockbox.getAllReplaceLocksForDatasource(datasource); +- segments.forEach(segment -> { +- locks.forEach(lock -> { +- if (lock.getGranularity() == LockGranularity.TIME_CHUNK) { +- final TimeChunkLock timeChunkLock = (TimeChunkLock) lock; +- if (timeChunkLock.getInterval().contains(segment.getInterval()) +- && timeChunkLock.getDataSource().equals(segment.getDataSource())) { +- found.add(lock); +- } ++ // Identify unique segment intervals ++ final Map> intervalToSegments = new HashMap<>(); ++ segments.forEach( ++ segment -> intervalToSegments.computeIfAbsent( ++ segment.getInterval(), interval -> new ArrayList<>() ++ ).add(segment) ++ ); ++ ++ final Set replaceLocks = taskLockbox.getAllReplaceLocksForDatasource(datasource).stream() ++ .map(TaskLocks::toLockInfo) ++ .collect(Collectors.toSet()); ++ ++ final Map segmentToReplaceLock = new HashMap<>(); ++ ++ intervalToSegments.forEach((interval, segmentsForInterval) -> { ++ // For each interval, find the lock that covers it, if any ++ for (TaskLockInfo lock : replaceLocks) { ++ if (lock.getInterval().contains(interval)) { ++ segmentsForInterval.forEach(s -> segmentToReplaceLock.put(s, lock)); ++ return; + } +- }); ++ } + }); +- return found; ++ ++ return segmentToReplaceLock; ++ } ++ ++ public static TaskLockInfo toLockInfo(TaskLock taskLock) ++ { ++ return new TaskLockInfo(taskLock.getInterval(), taskLock.getVersion()); + } + + public static List findLocksForSegments( +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java +index cd7a52f772..20f7584c8e 100644 +--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java +@@ -33,6 +33,7 @@ import org.apache.druid.server.security.ForbiddenException; + import org.apache.druid.server.security.Resource; + import org.apache.druid.server.security.ResourceAction; + import org.apache.druid.server.security.ResourceType; ++import org.apache.druid.timeline.DataSegment; + import org.apache.druid.utils.CircularBuffer; + import org.joda.time.DateTime; + +@@ -141,4 +142,14 @@ public class IndexTaskUtils + metricBuilder.setDimension(DruidMetrics.TASK_ID, taskStatus.getId()); + metricBuilder.setDimension(DruidMetrics.TASK_STATUS, taskStatus.getStatusCode().toString()); + } ++ ++ public static void setSegmentDimensions( ++ ServiceMetricEvent.Builder metricBuilder, ++ DataSegment segment ++ ) ++ { ++ final String partitionType = segment.getShardSpec() == null ? null : segment.getShardSpec().getType(); ++ metricBuilder.setDimension(DruidMetrics.PARTITIONING_TYPE, partitionType); ++ metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); ++ } + } +diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +index db94038e07..eaeb74866f 100644 +--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java ++++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +@@ -1523,14 +1523,14 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand + } + + @Override +- public SegmentPublishResult commitSegments( ++ public SegmentPublishResult commitSegmentsAndMetadata( + Set segments, + Set segmentsToDrop, + DataSourceMetadata startMetadata, + DataSourceMetadata endMetadata + ) throws IOException + { +- SegmentPublishResult result = super.commitSegments(segments, segmentsToDrop, startMetadata, endMetadata); ++ SegmentPublishResult result = super.commitSegmentsAndMetadata(segments, segmentsToDrop, startMetadata, endMetadata); + + Assert.assertFalse( + "Segment latch not initialized, did you forget to call expectPublishSegments?", +diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +index 950983d77b..d22769c1f7 100644 +--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java ++++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +@@ -29,7 +29,6 @@ import org.apache.druid.data.input.impl.JsonInputFormat; + import org.apache.druid.data.input.impl.LocalInputSource; + import org.apache.druid.indexer.TaskState; + import org.apache.druid.indexing.common.LockGranularity; +-import org.apache.druid.indexing.common.TaskLockType; + import org.apache.druid.indexing.common.TaskToolbox; + import org.apache.druid.indexing.common.actions.TaskActionClient; + import org.apache.druid.indexing.common.task.Tasks; +diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +index e58df94ef8..16c775b01b 100644 +--- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java ++++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +@@ -173,8 +173,7 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto + Set segments, + DataSourceMetadata oldCommitMetadata, + DataSourceMetadata newCommitMetadata, +- @Nullable Map segmentLockMap, +- @Nullable Set taskLockInfos ++ @Nullable Map segmentLockMap + ) + { + // Don't actually compare metadata, just do it! +@@ -182,7 +181,7 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto + } + + @Override +- public SegmentPublishResult commitSegments( ++ public SegmentPublishResult commitSegmentsAndMetadata( + Set segments, + Set segmentsToDrop, + @Nullable DataSourceMetadata startMetadata, +diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java b/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java +index a6955ece8b..26fc4cc9db 100644 +--- a/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java ++++ b/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java +@@ -26,7 +26,7 @@ import java.util.Set; + + /** + * Commit metadata for a dataSource. Used by +- * {@link IndexerMetadataStorageCoordinator#commitSegments(Set, Set, DataSourceMetadata, DataSourceMetadata)} ++ * {@link IndexerMetadataStorageCoordinator#commitSegmentsAndMetadata(Set, Set, DataSourceMetadata, DataSourceMetadata)} + * to provide metadata transactions for segment inserts. + * + * Two metadata instances can be added together, and any conflicts are resolved in favor of the right-hand side. +diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +index e5e09269b9..0edadd9605 100644 +--- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java ++++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +@@ -254,7 +254,7 @@ public interface IndexerMetadataStorageCoordinator + * commit metadata. + * + * If segmentsToDrop is not null and not empty, this insertion will be atomic with a insert-and-drop on inserting +- * {@param segments} and dropping {@param segmentsToDrop} ++ * {@param segments} and dropping {@param segmentsToDrop}. + * + * @param segments set of segments to add, must all be from the same dataSource + * @param segmentsToDrop set of segments to drop, must all be from the same dataSource +@@ -272,7 +272,7 @@ public interface IndexerMetadataStorageCoordinator + * @throws IllegalArgumentException if startMetadata and endMetadata are not either both null or both non-null + * @throws RuntimeException if the state of metadata storage after this call is unknown + */ +- SegmentPublishResult commitSegments( ++ SegmentPublishResult commitSegmentsAndMetadata( + Set segments, + Set segmentsToDrop, + @Nullable DataSourceMetadata startMetadata, +@@ -283,8 +283,7 @@ public interface IndexerMetadataStorageCoordinator + Set segments, + @Nullable DataSourceMetadata startMetadata, + @Nullable DataSourceMetadata endMetadata, +- @Nullable Map segmentLockMap, +- @Nullable Set taskLockInfos ++ @Nullable Map segmentLockMap + ) throws IOException; + + SegmentPublishResult commitReplaceSegments( +diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +index 5ee437914a..5e26bc33b6 100644 +--- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java ++++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +@@ -309,7 +309,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor + @Override + public Set commitSegments(final Set segments) throws IOException + { +- final SegmentPublishResult result = commitSegments(segments, null, null, null); ++ final SegmentPublishResult result = commitSegmentsAndMetadata(segments, null, null, null); + + // Metadata transaction cannot fail because we are not trying to do one. + if (!result.isSuccess()) { +@@ -320,7 +320,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor + } + + @Override +- public SegmentPublishResult commitSegments( ++ public SegmentPublishResult commitSegmentsAndMetadata( + final Set segments, + final Set segmentsToDrop, + @Nullable final DataSourceMetadata startMetadata, +@@ -459,41 +459,31 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor + + try { + return connector.retryTransaction( +- new TransactionCallback() +- { +- @Override +- public SegmentPublishResult inTransaction( +- final Handle handle, +- final TransactionStatus transactionStatus +- ) throws Exception +- { +- // Set definitelyNotUpdated back to false upon retrying. +- definitelyNotUpdated.set(false); +- ++ (handle, transactionStatus) -> { ++ // Set definitelyNotUpdated back to false upon retrying. ++ definitelyNotUpdated.set(false); + +- if (segmentsToDrop != null && !segmentsToDrop.isEmpty()) { +- final DataStoreMetadataUpdateResult result = dropSegmentsWithHandle( +- handle, +- segmentsToDrop, +- dataSource +- ); +- if (result.isFailed()) { +- // Metadata store was definitely not updated. +- transactionStatus.setRollbackOnly(); +- definitelyNotUpdated.set(true); ++ if (segmentsToDrop != null && !segmentsToDrop.isEmpty()) { ++ final DataStoreMetadataUpdateResult result = dropSegmentsWithHandle( ++ handle, ++ segmentsToDrop, ++ dataSource ++ ); ++ if (result.isFailed()) { ++ // Metadata store was definitely not updated. ++ transactionStatus.setRollbackOnly(); ++ definitelyNotUpdated.set(true); + +- if (result.canRetry()) { +- throw new RetryTransactionException(result.getErrorMsg()); +- } else { +- throw new RuntimeException(result.getErrorMsg()); +- } ++ if (result.canRetry()) { ++ throw new RetryTransactionException(result.getErrorMsg()); ++ } else { ++ throw new RuntimeException(result.getErrorMsg()); + } + } +- +- final Set inserted = commitReplaceSegmentBatch(handle, newSegments, usedSegments, taskLockInfos); +- +- return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); + } ++ ++ final Set inserted = commitReplaceSegmentBatch(handle, newSegments, usedSegments, taskLockInfos); ++ return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); + }, + 3, + getSqlMetadataMaxRetry() +@@ -514,43 +504,28 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor + final Set segments, + @Nullable final DataSourceMetadata startMetadata, + @Nullable DataSourceMetadata endMetadata, +- @Nullable Map segmentLockMap, +- @Nullable Set taskLockInfos ++ @Nullable Map segmentLockMap + ) + { + if (segments.isEmpty()) { +- throw new IllegalArgumentException("segment set must not be empty"); ++ throw new IllegalArgumentException("No segments to append"); + } + + final String dataSource = segments.iterator().next().getDataSource(); + for (DataSegment segment : segments) { + if (!dataSource.equals(segment.getDataSource())) { +- throw new IllegalArgumentException("segments must all be from the same dataSource"); ++ throw new IllegalArgumentException("All segments to append must belong to the same dataSource"); + } + } + + if ((startMetadata == null && endMetadata != null) || (startMetadata != null && endMetadata == null)) { +- throw new IllegalArgumentException("start/end metadata pair must be either null or non-null"); ++ throw new IllegalArgumentException("Start and end metadata must either be both null or both non-null"); + } + + // Find which segments are used (i.e. not overshadowed). + Set newSegments = new HashSet<>(segments); + final Map> segmentToNewMetadataMap = connector.retryTransaction( +- new TransactionCallback>>() +- { +- @Override +- public Map> inTransaction( +- final Handle handle, +- final TransactionStatus transactionStatus +- ) throws Exception +- { +- return allocateNewSegmentIds( +- handle, +- dataSource, +- segments +- ); +- } +- }, ++ (handle, transactionStatus) -> allocateNewSegmentIds(handle, dataSource, segments), + 0, + SQLMetadataConnector.DEFAULT_MAX_TRIES + ); +@@ -572,49 +547,36 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor + } + Set usedSegments = new HashSet<>(newSegments); + +- +- + final AtomicBoolean definitelyNotUpdated = new AtomicBoolean(false); +- + try { + return connector.retryTransaction( +- new TransactionCallback() +- { +- @Override +- public SegmentPublishResult inTransaction( +- final Handle handle, +- final TransactionStatus transactionStatus +- ) throws Exception +- { +- // Set definitelyNotUpdated back to false upon retrying. +- definitelyNotUpdated.set(false); ++ (handle, transactionStatus) -> { ++ // Set definitelyNotUpdated back to false upon retrying. ++ definitelyNotUpdated.set(false); + +- if (startMetadata != null) { +- final DataStoreMetadataUpdateResult result = updateDataSourceMetadataWithHandle( +- handle, +- dataSource, +- startMetadata, +- endMetadata +- ); ++ if (startMetadata != null) { ++ final DataStoreMetadataUpdateResult result = updateDataSourceMetadataWithHandle( ++ handle, ++ dataSource, ++ startMetadata, ++ endMetadata ++ ); + +- if (result.isFailed()) { +- // Metadata was definitely not updated. +- transactionStatus.setRollbackOnly(); +- definitelyNotUpdated.set(true); ++ if (result.isFailed()) { ++ // Metadata was definitely not updated. ++ transactionStatus.setRollbackOnly(); ++ definitelyNotUpdated.set(true); + +- if (result.canRetry()) { +- throw new RetryTransactionException(result.getErrorMsg()); +- } else { +- throw new RuntimeException(result.getErrorMsg()); +- } ++ if (result.canRetry()) { ++ throw new RetryTransactionException(result.getErrorMsg()); ++ } else { ++ throw new RuntimeException(result.getErrorMsg()); + } + } +- +- +- final Set inserted = commitAppendSegmentBatch(handle, newSegments, usedSegments, segmentLockMap); +- +- return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); + } ++ ++ final Set inserted = commitAppendSegmentBatch(handle, newSegments, usedSegments, segmentLockMap); ++ return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); + }, + 3, + getSqlMetadataMaxRetry() +@@ -1299,7 +1261,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor + } + for (DataSegment segment : intervalToSegments.get(interval)) { + SegmentCreateRequest request = new SegmentCreateRequest( +- segment.getId().toString() + version, ++ segment.getId() + version, + null, + version, + NumberedPartialShardSpec.instance() +@@ -1867,7 +1829,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor + for (Map.Entry entry : batch) { + batchMap.put(entry.getKey(), entry.getValue()); + } +- List oldSegments = retrieveSegments(handle, batchMap.keySet()); ++ List oldSegments = retrieveSegmentsById(handle, batchMap.keySet()); + for (DataSegment oldSegment : oldSegments) { + Interval newInterval = oldSegment.getInterval(); + for (DataSegment segment : segments) { +@@ -2007,122 +1969,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor + DataSegment segment = entry.getKey(); + TaskLockInfo lock = entry.getValue(); + appendBatch.add() +- .bind("id", segment.getId().toString() + ":" + lock.hashCode()) +- .bind("dataSource", segment.getDataSource()) +- .bind("start", lock.getInterval().getStartMillis()) +- .bind("end", lock.getInterval().getEndMillis()) +- .bind("segment_id", segment.getId().toString()) +- .bind("lock_version", lock.getVersion()); +- } +- final int[] affectedAppendRows = appendBatch.execute(); +- final boolean succeeded = Arrays.stream(affectedAppendRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); +- if (!succeeded) { +- final List failedToForward = IntStream.range(0, partition.size()) +- .filter(i -> affectedAppendRows[i] != 1) +- .mapToObj(partition::get) +- .map(x -> x.getKey()) +- .collect(Collectors.toList()); +- throw new ISE( +- "Failed to forward appended segments to DB: %s", +- SegmentUtils.commaSeparatedIdentifiers(failedToForward) +- ); +- } +- } +- } +- catch (Exception e) { +- log.errorSegments(segments, "Exception inserting segment metadata"); +- throw e; +- } +- +- return toInsertSegments; +- } +- +- private Set commitSegmentBatch( +- final Handle handle, +- final Set segments, +- final Set usedSegments, +- @Nullable Map appendSegmentLockMap, +- @Nullable Set replaceLocks, +- boolean append +- ) throws IOException +- { +- final Set toInsertSegments = new HashSet<>(); +- try { +- Set existedSegments = segmentExistsBatch(handle, segments); +- log.info("Found these segments already exist in DB: %s", existedSegments); +- for (DataSegment segment : segments) { +- if (!existedSegments.contains(segment.getId().toString())) { +- toInsertSegments.add(segment); +- } +- } +- +- // SELECT -> INSERT can fail due to races; callers must be prepared to retry. +- // Avoiding ON DUPLICATE KEY since it's not portable. +- // Avoiding try/catch since it may cause inadvertent transaction-splitting. +- final List> partitionedSegments = Lists.partition( +- new ArrayList<>(toInsertSegments), +- MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE +- ); +- +- PreparedBatch preparedBatch = handle.prepareBatch( +- StringUtils.format( +- "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload) " +- + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", +- dbTables.getSegmentsTable(), +- connector.getQuoteString() +- ) +- ); +- for (List partition : partitionedSegments) { +- for (DataSegment segment : partition) { +- preparedBatch.add() +- .bind("id", segment.getId().toString()) +- .bind("dataSource", segment.getDataSource()) +- .bind("created_date", DateTimes.nowUtc().toString()) +- .bind("start", segment.getInterval().getStart().toString()) +- .bind("end", segment.getInterval().getEnd().toString()) +- .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) +- .bind("version", segment.getVersion()) +- .bind("used", usedSegments.contains(segment)) +- .bind("payload", jsonMapper.writeValueAsBytes(segment)); +- } +- final int[] affectedInsertRows = preparedBatch.execute(); +- +- final boolean succeeded = Arrays.stream(affectedInsertRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); +- if (succeeded) { +- log.infoSegments(partition, "Published segments to DB"); +- } else { +- final List failedToPublish = IntStream.range(0, partition.size()) +- .filter(i -> affectedInsertRows[i] != 1) +- .mapToObj(partition::get) +- .collect(Collectors.toList()); +- throw new ISE( +- "Failed to publish segments to DB: %s", +- SegmentUtils.commaSeparatedIdentifiers(failedToPublish) +- ); +- } +- } +- +- PreparedBatch appendBatch = handle.prepareBatch( +- StringUtils.format( +- "INSERT INTO %1$s (id, dataSource, start, %2$send%2$s, segment_id, lock_version) " +- + "VALUES (:id, :dataSource, :start, :end, :segment_id, :lock_version)", +- dbTables.getSegmentVersionsTable(), +- connector.getQuoteString() +- ) +- ); +- if (appendSegmentLockMap == null) { +- appendSegmentLockMap = new HashMap<>(); +- } +- final List>> appendSegmentPartitions = Lists.partition( +- new ArrayList<>(appendSegmentLockMap.entrySet()), +- MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE +- ); +- for (List> partition : appendSegmentPartitions) { +- for (Map.Entry entry : partition) { +- DataSegment segment = entry.getKey(); +- TaskLockInfo lock = entry.getValue(); +- appendBatch.add() +- .bind("id", segment.getId().toString() + ":" + lock.hashCode()) ++ .bind("id", segment.getId() + ":" + lock.hashCode()) + .bind("dataSource", segment.getDataSource()) + .bind("start", lock.getInterval().getStartMillis()) + .bind("end", lock.getInterval().getEndMillis()) +@@ -2143,74 +1990,6 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor + ); + } + } +- +- Map segmentsToBeForwarded = new HashMap<>(); +- if (!append) { +- segmentsToBeForwarded = getAppendedSegmentIds( +- handle, +- segments.iterator().next().getDataSource(), +- replaceLocks +- ); +- } +- final int numCorePartitions = segments.size(); +- int partitionNum = segments.size(); +- final List>> forwardSegmentsBatch = Lists.partition( +- new ArrayList<>(segmentsToBeForwarded.entrySet()), +- MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE +- ); +- for (List> batch : forwardSegmentsBatch) { +- Map batchMap = new HashMap<>(); +- for (Map.Entry entry : batch) { +- batchMap.put(entry.getKey(), entry.getValue()); +- } +- List oldSegments = retrieveSegments(handle, batchMap.keySet()); +- for (DataSegment oldSegment : oldSegments) { +- Interval newInterval = oldSegment.getInterval(); +- for (DataSegment segment : segments) { +- if (segment.getInterval().overlaps(newInterval)) { +- if (segment.getInterval().contains(newInterval)) { +- newInterval = segment.getInterval(); +- } else { +- throw new ISE("Incompatible segment intervals for commit: [%s] and [%s].", +- newInterval, +- segment.getInterval() +- ); +- } +- } +- } +- TaskLockInfo lock = batchMap.get(oldSegment.getId().toString()); +- ShardSpec shardSpec = new NumberedShardSpec(partitionNum++, numCorePartitions); +- DataSegment newSegment = new DataSegment( +- oldSegment.getDataSource(), +- newInterval, +- lock.getVersion(), +- oldSegment.getLoadSpec(), +- oldSegment.getDimensions(), +- oldSegment.getMetrics(), +- shardSpec, +- oldSegment.getBinaryVersion(), +- oldSegment.getSize() +- ); +- preparedBatch.add() +- .bind("id", newSegment.getId().toString()) +- .bind("dataSource", newSegment.getDataSource()) +- .bind("created_date", DateTimes.nowUtc().toString()) +- .bind("start", newSegment.getInterval().getStart().toString()) +- .bind("end", newSegment.getInterval().getEnd().toString()) +- .bind("partitioned", (newSegment.getShardSpec() instanceof NoneShardSpec) ? false : true) +- .bind("version", newSegment.getVersion()) +- .bind("used", true) +- .bind("payload", jsonMapper.writeValueAsBytes(newSegment)); +- } +- final int[] affectedInsertRows = preparedBatch.execute(); +- +- final boolean succeeded = Arrays.stream(affectedInsertRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); +- if (succeeded) { +- log.info("Published segments with updated metadata to DB"); +- } else { +- throw new ISE("Failed to update segment metadatas in DB"); +- } +- } + } + catch (Exception e) { + log.errorSegments(segments, "Exception inserting segment metadata"); +@@ -2220,37 +1999,22 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor + return toInsertSegments; + } + +- private List retrieveSegments(final Handle handle, final Set segmentIds) ++ private List retrieveSegmentsById(Handle handle, Set segmentIds) + { +- final StringBuilder sb = new StringBuilder(); +- sb.append("SELECT payload FROM %s WHERE id in ("); +- +- List segmentIdList = new ArrayList<>(segmentIds); +- int n = segmentIdList.size(); +- for (int i = 0; i < n; i++) { +- sb.append("'"); +- sb.append(segmentIdList.get(i)); +- sb.append("'"); +- if (i < n - 1) { +- sb.append(", "); +- } +- } +- +- sb.append(")"); +- +- final Query> sql = handle +- .createQuery(StringUtils.format(sb.toString(), dbTables.getSegmentsTable())) +- .setFetchSize(connector.getStreamingFetchSize()); ++ final String segmentIdCsv = segmentIds.stream().map(id -> "'" + id + "'") ++ .collect(Collectors.joining(",")); ++ final Query> query = handle.createQuery( ++ StringUtils.format( ++ "SELECT payload FROM %s WHERE id in (%s)", ++ segmentIdCsv, dbTables.getSegmentsTable() ++ ) ++ ).setFetchSize(connector.getStreamingFetchSize()); + +- final ResultIterator resultIterator = +- sql.map((index, r, ctx) -> JacksonUtils.readValue(jsonMapper, r.getBytes(1), DataSegment.class)) +- .iterator(); ++ ResultIterator resultIterator = query.map( ++ (index, r, ctx) -> JacksonUtils.readValue(jsonMapper, r.getBytes(1), DataSegment.class) ++ ).iterator(); + +- List retVal = new ArrayList<>(); +- while (resultIterator.hasNext()) { +- retVal.add(resultIterator.next()); +- } +- return retVal; ++ return Lists.newArrayList(resultIterator); + } + + private Map getAppendedSegmentIds( +@@ -2383,7 +2147,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor + * + * @return SUCCESS if dataSource metadata was updated from matching startMetadata to matching endMetadata, FAILURE or + * TRY_AGAIN if it definitely was not updated. This guarantee is meant to help +- * {@link #commitSegments(Set, Set, DataSourceMetadata, DataSourceMetadata)} ++ * {@link #commitSegmentsAndMetadata(Set, Set, DataSourceMetadata, DataSourceMetadata)} + * achieve its own guarantee. + * + * @throws RuntimeException if state is unknown after this call +@@ -2513,7 +2277,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor + * + * @return SUCCESS if segment was marked unused, FAILURE or + * TRY_AGAIN if it definitely was not updated. This guarantee is meant to help +- * {@link #commitSegments(Set, Set, DataSourceMetadata, DataSourceMetadata)} ++ * {@link #commitSegmentsAndMetadata(Set, Set, DataSourceMetadata, DataSourceMetadata)} + * achieve its own guarantee. + * + * @throws RuntimeException if state is unknown after this call +diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +index a802c3c655..669ffefae2 100644 +--- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java ++++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +@@ -346,7 +346,7 @@ public abstract class SQLMetadataConnector implements MetadataStorageConnector + ); + } + +- public void createSegmentVersionTable(final String tableName) ++ private void createSegmentVersionTable(final String tableName) + { + createTable( + tableName, +diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +index eb55f04e20..8845a3beef 100644 +--- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java ++++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +@@ -558,7 +558,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest + public void testTransactionalAnnounceSuccess() throws IOException + { + // Insert first segment. +- final SegmentPublishResult result1 = coordinator.commitSegments( ++ final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( + ImmutableSet.of(defaultSegment), + ImmutableSet.of(), + new ObjectMetadata(null), +@@ -577,7 +577,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest + ); + + // Insert second segment. +- final SegmentPublishResult result2 = coordinator.commitSegments( ++ final SegmentPublishResult result2 = coordinator.commitSegmentsAndMetadata( + ImmutableSet.of(defaultSegment2), + ImmutableSet.of(), + new ObjectMetadata(ImmutableMap.of("foo", "bar")), +@@ -634,7 +634,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest + }; + + // Insert first segment. +- final SegmentPublishResult result1 = failOnceCoordinator.commitSegments( ++ final SegmentPublishResult result1 = failOnceCoordinator.commitSegmentsAndMetadata( + ImmutableSet.of(defaultSegment), + ImmutableSet.of(), + new ObjectMetadata(null), +@@ -656,7 +656,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest + attemptCounter.set(0); + + // Insert second segment. +- final SegmentPublishResult result2 = failOnceCoordinator.commitSegments( ++ final SegmentPublishResult result2 = failOnceCoordinator.commitSegmentsAndMetadata( + ImmutableSet.of(defaultSegment2), + ImmutableSet.of(), + new ObjectMetadata(ImmutableMap.of("foo", "bar")), +@@ -687,7 +687,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest + @Test + public void testTransactionalAnnounceFailDbNullWantNotNull() throws IOException + { +- final SegmentPublishResult result1 = coordinator.commitSegments( ++ final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( + ImmutableSet.of(defaultSegment), + ImmutableSet.of(), + new ObjectMetadata(ImmutableMap.of("foo", "bar")), +@@ -721,7 +721,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest + .build(); + Set dropSegments = ImmutableSet.of(existingSegment1, existingSegment2, dataSegmentBar); + +- final SegmentPublishResult result1 = coordinator.commitSegments( ++ final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( + SEGMENTS, + dropSegments, + null, +@@ -750,7 +750,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest + retrieveUsedSegmentIds() + ); + +- final SegmentPublishResult result1 = coordinator.commitSegments( ++ final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( + SEGMENTS, + ImmutableSet.of(existingSegment1, existingSegment2), + null, +@@ -788,7 +788,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest + ); + + Set dropSegments = ImmutableSet.of(existingSegment1, defaultSegment4); +- final SegmentPublishResult result1 = coordinator.commitSegments( ++ final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( + SEGMENTS, + dropSegments, + null, +@@ -809,7 +809,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest + @Test + public void testTransactionalAnnounceFailDbNotNullWantNull() throws IOException + { +- final SegmentPublishResult result1 = coordinator.commitSegments( ++ final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( + ImmutableSet.of(defaultSegment), + ImmutableSet.of(), + new ObjectMetadata(null), +@@ -817,7 +817,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest + ); + Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(defaultSegment)), result1); + +- final SegmentPublishResult result2 = coordinator.commitSegments( ++ final SegmentPublishResult result2 = coordinator.commitSegmentsAndMetadata( + ImmutableSet.of(defaultSegment2), + ImmutableSet.of(), + new ObjectMetadata(null), +@@ -835,7 +835,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest + @Test + public void testTransactionalAnnounceFailDbNotNullWantDifferent() throws IOException + { +- final SegmentPublishResult result1 = coordinator.commitSegments( ++ final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( + ImmutableSet.of(defaultSegment), + ImmutableSet.of(), + new ObjectMetadata(null), +@@ -843,7 +843,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest + ); + Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(defaultSegment)), result1); + +- final SegmentPublishResult result2 = coordinator.commitSegments( ++ final SegmentPublishResult result2 = coordinator.commitSegmentsAndMetadata( + ImmutableSet.of(defaultSegment2), + ImmutableSet.of(), + new ObjectMetadata(ImmutableMap.of("foo", "qux")), +@@ -1391,7 +1391,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest + @Test + public void testDeleteDataSourceMetadata() throws IOException + { +- coordinator.commitSegments( ++ coordinator.commitSegmentsAndMetadata( + ImmutableSet.of(defaultSegment), + ImmutableSet.of(), + new ObjectMetadata(null), +@@ -2347,7 +2347,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest + @Test + public void testRemoveDataSourceMetadataOlderThanDatasourceActiveShouldNotBeDeleted() throws Exception + { +- coordinator.commitSegments( ++ coordinator.commitSegmentsAndMetadata( + ImmutableSet.of(defaultSegment), + ImmutableSet.of(), + new ObjectMetadata(null), +@@ -2376,7 +2376,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest + @Test + public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveAndOlderThanTimeShouldBeDeleted() throws Exception + { +- coordinator.commitSegments( ++ coordinator.commitSegmentsAndMetadata( + ImmutableSet.of(defaultSegment), + ImmutableSet.of(), + new ObjectMetadata(null), +@@ -2402,7 +2402,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest + public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveButNotOlderThanTimeShouldNotBeDeleted() + throws Exception + { +- coordinator.commitSegments( ++ coordinator.commitSegmentsAndMetadata( + ImmutableSet.of(defaultSegment), + ImmutableSet.of(), + new ObjectMetadata(null), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java deleted file mode 100644 index cd468e0f3706..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ConcurrentReplaceAndAppendTest.java +++ /dev/null @@ -1,293 +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.google.common.collect.ImmutableList; -import com.google.common.collect.Sets; -import org.apache.druid.indexing.common.MultipleFileTaskReportFileWriter; -import org.apache.druid.indexing.common.TaskStorageDirTracker; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.TaskToolboxFactory; -import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; -import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.actions.TaskActionClientFactory; -import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.config.TaskConfigBuilder; -import org.apache.druid.indexing.common.task.batch.parallel.ActionsTestTask; -import org.apache.druid.indexing.overlord.Segments; -import org.apache.druid.indexing.overlord.TaskQueue; -import org.apache.druid.indexing.overlord.TaskRunner; -import org.apache.druid.indexing.overlord.TestTaskToolboxFactory; -import org.apache.druid.indexing.overlord.ThreadingTaskRunner; -import org.apache.druid.indexing.overlord.config.DefaultTaskConfig; -import org.apache.druid.indexing.overlord.config.TaskLockConfig; -import org.apache.druid.indexing.overlord.config.TaskQueueConfig; -import org.apache.druid.indexing.worker.config.WorkerConfig; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.column.ColumnConfig; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.server.DruidNode; -import org.apache.druid.server.metrics.NoopServiceEmitter; -import org.apache.druid.tasklogs.NoopTaskLogs; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; -import org.joda.time.Interval; -import org.joda.time.Period; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Set; - -public class ConcurrentReplaceAndAppendTest extends IngestionTestBase -{ - private static final WorkerConfig WORKER_CONFIG = new WorkerConfig().setCapacity(10); - - private TaskQueue taskQueue; - private TaskRunner taskRunner; - private TaskActionClientFactory taskActionClientFactory; - private TaskActionClient dummyTaskActionClient; - private final List runningTasks = new ArrayList<>(); - - @Before - public void setup() - { - final TaskConfig taskConfig = new TaskConfigBuilder().build(); - taskActionClientFactory = createActionClientFactory(); - dummyTaskActionClient = taskActionClientFactory.create(NoopTask.create()); - taskRunner = new ThreadingTaskRunner( - createToolboxFactory(taskConfig, taskActionClientFactory), - taskConfig, - WORKER_CONFIG, - new NoopTaskLogs(), - getObjectMapper(), - new TestAppenderatorsManager(), - new MultipleFileTaskReportFileWriter(), - new DruidNode("middleManager", "host", false, 8091, null, true, false), - TaskStorageDirTracker.fromConfigs(WORKER_CONFIG, taskConfig) - ); - taskQueue = new TaskQueue( - new TaskLockConfig(), - new TaskQueueConfig(null, new Period(0L), null, null, null), - new DefaultTaskConfig(), - getTaskStorage(), - taskRunner, - taskActionClientFactory, - getLockbox(), - new NoopServiceEmitter() - ); - runningTasks.clear(); - taskQueue.start(); - } - - @After - public void tearDown() - { - for (Task task : runningTasks) { - if (task instanceof ActionsTestTask) { - ((ActionsTestTask) task).finishRunAndGetStatus(); - } - } - } - - @Test - public void testAppendSegmentGetsUpgraded() - { - final Interval year2023 = Intervals.of("2023/2024"); - - // Commit initial segments for v0 - final ActionsTestTask replaceTask0 = createAndStartTask(); - final String v0 = replaceTask0.acquireReplaceLockOn(year2023).getVersion(); - - final DataSegment segmentV00 = createSegment(year2023, v0); - replaceTask0.commitReplaceSegments(segmentV00); - replaceTask0.finishRunAndGetStatus(); - verifyIntervalHasUsedSegments(year2023, segmentV00); - verifyIntervalHasVisibleSegments(year2023, segmentV00); - - // Allocate an append segment for v0 - final ActionsTestTask appendTask0 = createAndStartTask(); - appendTask0.acquireAppendLockOn(year2023); - final SegmentIdWithShardSpec pendingSegmentV01 - = appendTask0.allocateSegmentForTimestamp(year2023.getStart(), Granularities.YEAR); - Assert.assertEquals(segmentV00.getVersion(), pendingSegmentV01.getVersion()); - - // Commit replace segment for v1 - final ActionsTestTask replaceTask1 = createAndStartTask(); - final String v1 = replaceTask1.acquireReplaceLockOn(year2023).getVersion(); - - final DataSegment segmentV10 = DataSegment.builder(segmentV00).version(v1).build(); - replaceTask1.commitReplaceSegments(segmentV10); - replaceTask1.finishRunAndGetStatus(); - verifyIntervalHasUsedSegments(year2023, segmentV00, segmentV10); - verifyIntervalHasVisibleSegments(year2023, segmentV10); - - final ActionsTestTask replaceTask2 = createAndStartTask(); - final String v2 = replaceTask2.acquireReplaceLockOn(year2023).getVersion(); - - // Commit append segment v0 and verify that it gets upgraded to v1 - final DataSegment segmentV01 = asSegment(pendingSegmentV01); - final DataSegment segmentV11 = DataSegment.builder(segmentV01).version(v1).build(); - Set appendedSegments = appendTask0.commitAppendSegments(segmentV01).getSegments(); - Assert.assertEquals(Sets.newHashSet(segmentV01, segmentV11), appendedSegments); - - appendTask0.finishRunAndGetStatus(); - verifyIntervalHasUsedSegments( - year2023, - segmentV00, segmentV01, segmentV10, segmentV11 - ); - verifyIntervalHasVisibleSegments(year2023, segmentV10, segmentV11); - - // Commit replace segment v2 and verify that append segment gets upgraded to v2 - final DataSegment segmentV20 = DataSegment.builder(segmentV00).version(v2).build(); - replaceTask2.commitReplaceSegments(segmentV20); - replaceTask2.finishRunAndGetStatus(); - - final DataSegment segmentV21 = DataSegment.builder(segmentV01).version(v2).build(); - verifyIntervalHasUsedSegments( - year2023, - segmentV00, segmentV01, segmentV10, segmentV11, segmentV20, segmentV21 - ); - verifyIntervalHasVisibleSegments(year2023, segmentV20, segmentV21); - } - - @Test - public void testRRAA_dailyReplaceDailyAppend() - { - final Interval firstOfJan = Intervals.of("2023-01-01/2023-01-02"); - - final ActionsTestTask replaceTask0 = createAndStartTask(); - final ActionsTestTask appendTask0 = createAndStartTask(); - - final String v0 = replaceTask0.acquireReplaceLockOn(firstOfJan).getVersion(); - - final DataSegment segmentV00 = createSegment(firstOfJan, v0); - - replaceTask0.commitReplaceSegments(segmentV00); - replaceTask0.finishRunAndGetStatus(); - verifyIntervalHasUsedSegments(firstOfJan, segmentV00); - - appendTask0.acquireAppendLockOn(firstOfJan); - final SegmentIdWithShardSpec pendingSegment - = appendTask0.allocateSegmentForTimestamp(firstOfJan.getStart(), Granularities.DAY); - Assert.assertEquals(segmentV00.getVersion(), pendingSegment.getVersion()); - - final DataSegment segmentV01 = asSegment(pendingSegment); - appendTask0.commitAppendSegments(segmentV01); - - verifyIntervalHasUsedSegments(firstOfJan, segmentV00, segmentV01); - verifyIntervalHasVisibleSegments(firstOfJan, segmentV00, segmentV01); - } - - private static DataSegment asSegment(SegmentIdWithShardSpec pendingSegment) - { - final SegmentId id = pendingSegment.asSegmentId(); - return new DataSegment( - id, - Collections.singletonMap(id.toString(), id.toString()), - Collections.emptyList(), - Collections.emptyList(), - pendingSegment.getShardSpec(), - null, - 0, - 0 - ); - } - - private void verifyIntervalHasUsedSegments(Interval interval, DataSegment... expectedSegments) - { - verifySegments(interval, Segments.INCLUDING_OVERSHADOWED, expectedSegments); - } - - private void verifyIntervalHasVisibleSegments(Interval interval, DataSegment... expectedSegments) - { - verifySegments(interval, Segments.ONLY_VISIBLE, expectedSegments); - } - - private void verifySegments(Interval interval, Segments visibility, DataSegment... expectedSegments) - { - try { - Collection allUsedSegments = dummyTaskActionClient.submit( - new RetrieveUsedSegmentsAction( - DS.WIKI, - null, - ImmutableList.of(interval), - visibility - ) - ); - Assert.assertEquals(Sets.newHashSet(expectedSegments), Sets.newHashSet(allUsedSegments)); - } - catch (IOException e) { - throw new ISE(e, "Error while fetching used segments in interval[%s]", interval); - } - } - - private TaskToolboxFactory createToolboxFactory( - TaskConfig taskConfig, - TaskActionClientFactory taskActionClientFactory - ) - { - TestTaskToolboxFactory.Builder builder = new TestTaskToolboxFactory.Builder() - .setConfig(taskConfig) - .setIndexIO(new IndexIO(getObjectMapper(), ColumnConfig.DEFAULT)) - .setTaskActionClientFactory(taskActionClientFactory); - return new TestTaskToolboxFactory(builder) - { - @Override - public TaskToolbox build(TaskConfig config, Task task) - { - return createTaskToolbox(config, task); - } - }; - } - - private DataSegment createSegment(Interval interval, String version) - { - return DataSegment.builder() - .dataSource(DS.WIKI) - .interval(interval) - .version(version) - .size(100) - .build(); - } - - private ActionsTestTask createAndStartTask() - { - ActionsTestTask task = new ActionsTestTask(DS.WIKI, taskActionClientFactory); - taskQueue.add(task); - runningTasks.add(task); - return task; - } - - private static class DS - { - static final String WIKI = "wiki"; - } - -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ActionsTestTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java similarity index 79% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ActionsTestTask.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java index 035d791238d3..be4b8d8228c3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ActionsTestTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java @@ -1,4 +1,23 @@ -package org.apache.druid.indexing.common.task.batch.parallel; +/* + * 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.concurrent; import com.google.common.collect.Sets; import org.apache.druid.indexing.common.LockGranularity; @@ -11,7 +30,6 @@ import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; -import org.apache.druid.indexing.common.task.CommandQueueTask; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularities; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandQueueTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/CommandQueueTask.java similarity index 82% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandQueueTask.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/CommandQueueTask.java index 46d8060fa557..08e2c18112f9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CommandQueueTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/CommandQueueTask.java @@ -1,9 +1,29 @@ -package org.apache.druid.indexing.common.task; +/* + * 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.concurrent; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; @@ -54,7 +74,7 @@ public TaskStatus finishRunAndGetStatus() finishRequested.set(true); } synchronized (queueNotification) { - queueNotification.notify(); + queueNotification.notifyAll(); } try { @@ -106,7 +126,7 @@ private void addToQueue(Command command) } synchronized (queueNotification) { - queueNotification.notify(); + queueNotification.notifyAll(); } } @@ -121,7 +141,7 @@ private V waitForCommandToFinish(Command command) } @Override - public TaskStatus runTask(TaskToolbox taskToolbox) throws Exception + public TaskStatus runTask(TaskToolbox taskToolbox) { TaskStatus status; try { @@ -161,7 +181,7 @@ public String getType() } @Override - public boolean isReady(TaskActionClient taskActionClient) throws Exception + public boolean isReady(TaskActionClient taskActionClient) { return true; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAppendTest.java new file mode 100644 index 000000000000..81daeaf8feba --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAppendTest.java @@ -0,0 +1,447 @@ +/* + * 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.concurrent; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; +import org.apache.druid.indexing.common.MultipleFileTaskReportFileWriter; +import org.apache.druid.indexing.common.TaskStorageDirTracker; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.TaskToolboxFactory; +import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.actions.TaskActionClientFactory; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.config.TaskConfigBuilder; +import org.apache.druid.indexing.common.task.IngestionTestBase; +import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.TestAppenderatorsManager; +import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.indexing.overlord.TaskQueue; +import org.apache.druid.indexing.overlord.TaskRunner; +import org.apache.druid.indexing.overlord.TestTaskToolboxFactory; +import org.apache.druid.indexing.overlord.ThreadingTaskRunner; +import org.apache.druid.indexing.overlord.config.DefaultTaskConfig; +import org.apache.druid.indexing.overlord.config.TaskLockConfig; +import org.apache.druid.indexing.overlord.config.TaskQueueConfig; +import org.apache.druid.indexing.worker.config.WorkerConfig; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.apache.druid.tasklogs.NoopTaskLogs; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +/** + * Contains tests to verify behaviour of concurrently running REPLACE and APPEND + * tasks on the same interval of a datasource. + *

+ * The tests verify the interleaving of the following actions: + *

    + *
  • LOCK: Acquisiting of a lock on an interval by a replace task
  • + *
  • ALLOCATE: Allocation of a pending segment by an append task
  • + *
  • REPLACE: Commit of segments created by a replace task
  • + *
  • APPEND: Commit of segments created by an append task
  • + *
+ */ +public class ConcurrentReplaceAppendTest extends IngestionTestBase +{ + /** + * The version used by append jobs when no previous replace job has run on an interval. + */ + private static final String SEGMENT_V0 = "1970-01-01T00:00:00.000Z"; + + private static final Interval YEAR_23 = Intervals.of("2023/2024"); + private static final Interval JAN_23 = Intervals.of("2023-01/2023-02"); + private static final Interval FIRST_OF_JAN_23 = Intervals.of("2023-01-01/2023-01-02"); + + private static final String WIKI = "wiki"; + + private TaskQueue taskQueue; + private TaskActionClientFactory taskActionClientFactory; + private TaskActionClient dummyTaskActionClient; + private final List runningTasks = new ArrayList<>(); + + private ActionsTestTask appendTask; + private ActionsTestTask replaceTask; + + @Before + public void setup() + { + final TaskConfig taskConfig = new TaskConfigBuilder().build(); + taskActionClientFactory = createActionClientFactory(); + dummyTaskActionClient = taskActionClientFactory.create(NoopTask.create()); + + final WorkerConfig workerConfig = new WorkerConfig().setCapacity(10); + TaskRunner taskRunner = new ThreadingTaskRunner( + createToolboxFactory(taskConfig, taskActionClientFactory), + taskConfig, + workerConfig, + new NoopTaskLogs(), + getObjectMapper(), + new TestAppenderatorsManager(), + new MultipleFileTaskReportFileWriter(), + new DruidNode("middleManager", "host", false, 8091, null, true, false), + TaskStorageDirTracker.fromConfigs(workerConfig, taskConfig) + ); + taskQueue = new TaskQueue( + new TaskLockConfig(), + new TaskQueueConfig(null, new Period(0L), null, null, null), + new DefaultTaskConfig(), + getTaskStorage(), + taskRunner, + taskActionClientFactory, + getLockbox(), + new NoopServiceEmitter() + ); + runningTasks.clear(); + taskQueue.start(); + + appendTask = createAndStartTask(); + replaceTask = createAndStartTask(); + } + + @After + public void tearDown() + { + for (ActionsTestTask task : runningTasks) { + task.finishRunAndGetStatus(); + } + } + + @Test + public void testLockReplaceAllocateAppend() + { + final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); + + final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + replaceTask.finishRunAndGetStatus(); + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); + + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(segmentV10.getVersion(), pendingSegment.getVersion()); + + final DataSegment segmentV11 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV11); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + } + + @Test + public void testLockAllocateAppendReplace() + { + final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); + + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23).getVersion(); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV01); + + final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + replaceTask.finishRunAndGetStatus(); + + // Verify that the segment appended to v0 gets upgraded to v1 + final DataSegment segmentV11 = DataSegment.builder(segmentV01) + .shardSpec(new NumberedShardSpec(1, 1)) + .version(v1).build(); + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + } + + @Test + public void testLockAllocateReplaceAppend() + { + final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); + + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23).getVersion(); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + + final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + replaceTask.finishRunAndGetStatus(); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); + + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); + + // Verify that the segment appended to v0 gets upgraded to v1 + final DataSegment segmentV11 = DataSegment.builder(segmentV01) + .shardSpec(new NumberedShardSpec(1, 1)) + .version(v1).build(); + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + } + + @Test + public void testAllocateLockReplaceAppend() + { + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23).getVersion(); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + + final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); + + final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + replaceTask.finishRunAndGetStatus(); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); + + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); + + // Verify that the segment appended to v0 gets upgraded to v1 + final DataSegment segmentV11 = DataSegment.builder(segmentV01) + .shardSpec(new NumberedShardSpec(1, 1)) + .version(v1).build(); + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + } + + @Test + public void testAllocateLockAppendReplace() + { + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23).getVersion(); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + + final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); + + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV01); + + final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + replaceTask.finishRunAndGetStatus(); + + // Verify that the segment appended to v0 gets upgraded to v1 + final DataSegment segmentV11 = DataSegment.builder(segmentV01) + .shardSpec(new NumberedShardSpec(1, 1)) + .version(v1).build(); + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + } + + @Test + public void testAllocateAppendLockReplace() + { + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23).getVersion(); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV01); + + final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); + + final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + replaceTask.finishRunAndGetStatus(); + + // Verify that the segment appended to v0 gets fully overshadowed + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01, segmentV10); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); + } + + @Test + public void testLockReplaceAllocateLockReplaceLockReplaceAppend() + { + // Commit initial segments for v1 + final ActionsTestTask replaceTask1 = createAndStartTask(); + final String v1 = replaceTask1.acquireReplaceLockOn(YEAR_23).getVersion(); + + final DataSegment segmentV10 = createSegment(YEAR_23, v1); + replaceTask1.commitReplaceSegments(segmentV10); + replaceTask1.finishRunAndGetStatus(); + verifyIntervalHasUsedSegments(YEAR_23, segmentV10); + verifyIntervalHasVisibleSegments(YEAR_23, segmentV10); + + // Allocate an append segment for v1 + final ActionsTestTask appendTask1 = createAndStartTask(); + appendTask1.acquireAppendLockOn(YEAR_23); + final SegmentIdWithShardSpec pendingSegmentV11 + = appendTask1.allocateSegmentForTimestamp(YEAR_23.getStart(), Granularities.YEAR); + Assert.assertEquals(segmentV10.getVersion(), pendingSegmentV11.getVersion()); + + // Commit replace segment for v2 + final ActionsTestTask replaceTask2 = createAndStartTask(); + final String v2 = replaceTask2.acquireReplaceLockOn(YEAR_23).getVersion(); + + final DataSegment segmentV20 = DataSegment.builder(segmentV10).version(v2).build(); + replaceTask2.commitReplaceSegments(segmentV20); + replaceTask2.finishRunAndGetStatus(); + verifyIntervalHasUsedSegments(YEAR_23, segmentV10, segmentV20); + verifyIntervalHasVisibleSegments(YEAR_23, segmentV20); + + final ActionsTestTask replaceTask3 = createAndStartTask(); + final String v3 = replaceTask3.acquireReplaceLockOn(YEAR_23).getVersion(); + + // Commit append segment to v1 and verify that it gets upgraded to v2 + final DataSegment segmentV11 = asSegment(pendingSegmentV11); + final DataSegment segmentV21 = DataSegment.builder(segmentV11).version(v2).build(); + Set appendedSegments = appendTask1.commitAppendSegments(segmentV11).getSegments(); + Assert.assertEquals(Sets.newHashSet(segmentV21, segmentV11), appendedSegments); + + appendTask1.finishRunAndGetStatus(); + verifyIntervalHasUsedSegments( + YEAR_23, + segmentV20, segmentV21, segmentV10, segmentV11 + ); + verifyIntervalHasVisibleSegments(YEAR_23, segmentV20, segmentV21); + + // Commit replace segment v2 and verify that append segment gets upgraded to v2 + final DataSegment segmentV30 = DataSegment.builder(segmentV20).version(v3).build(); + replaceTask3.commitReplaceSegments(segmentV30); + replaceTask3.finishRunAndGetStatus(); + + final DataSegment segmentV31 = DataSegment.builder(segmentV21).version(v3).build(); + verifyIntervalHasUsedSegments( + YEAR_23, + segmentV10, segmentV11, segmentV20, segmentV21, segmentV30, segmentV31 + ); + verifyIntervalHasVisibleSegments(YEAR_23, segmentV30, segmentV31); + } + + private static DataSegment asSegment(SegmentIdWithShardSpec pendingSegment) + { + final SegmentId id = pendingSegment.asSegmentId(); + return new DataSegment( + id, + Collections.singletonMap(id.toString(), id.toString()), + Collections.emptyList(), + Collections.emptyList(), + pendingSegment.getShardSpec(), + null, + 0, + 0 + ); + } + + private void verifyIntervalHasUsedSegments(Interval interval, DataSegment... expectedSegments) + { + verifySegments(interval, Segments.INCLUDING_OVERSHADOWED, expectedSegments); + } + + private void verifyIntervalHasVisibleSegments(Interval interval, DataSegment... expectedSegments) + { + verifySegments(interval, Segments.ONLY_VISIBLE, expectedSegments); + } + + private void verifySegments(Interval interval, Segments visibility, DataSegment... expectedSegments) + { + try { + Collection allUsedSegments = dummyTaskActionClient.submit( + new RetrieveUsedSegmentsAction( + WIKI, + null, + ImmutableList.of(interval), + visibility + ) + ); + Assert.assertEquals(Sets.newHashSet(expectedSegments), Sets.newHashSet(allUsedSegments)); + } + catch (IOException e) { + throw new ISE(e, "Error while fetching used segments in interval[%s]", interval); + } + } + + private TaskToolboxFactory createToolboxFactory( + TaskConfig taskConfig, + TaskActionClientFactory taskActionClientFactory + ) + { + TestTaskToolboxFactory.Builder builder = new TestTaskToolboxFactory.Builder() + .setConfig(taskConfig) + .setIndexIO(new IndexIO(getObjectMapper(), ColumnConfig.DEFAULT)) + .setTaskActionClientFactory(taskActionClientFactory); + return new TestTaskToolboxFactory(builder) + { + @Override + public TaskToolbox build(TaskConfig config, Task task) + { + return createTaskToolbox(config, task); + } + }; + } + + private DataSegment createSegment(Interval interval, String version) + { + return DataSegment.builder() + .dataSource(WIKI) + .interval(interval) + .version(version) + .size(100) + .build(); + } + + private ActionsTestTask createAndStartTask() + { + ActionsTestTask task = new ActionsTestTask(WIKI, taskActionClientFactory); + taskQueue.add(task); + runningTasks.add(task); + return task; + } + +} From 2bb3b796b1a2bf644da1e59bc1f01d0ce787e8ec Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 19 Sep 2023 12:42:31 +0530 Subject: [PATCH 31/43] Add tests for multiple granularities --- concurrent.diff | 1093 ----------------- .../task/concurrent/ActionsTestTask.java | 17 +- .../ConcurrentReplaceAppendTest.java | 337 ++++- 3 files changed, 338 insertions(+), 1109 deletions(-) delete mode 100644 concurrent.diff diff --git a/concurrent.diff b/concurrent.diff deleted file mode 100644 index 0b0796dc8b4f..000000000000 --- a/concurrent.diff +++ /dev/null @@ -1,1093 +0,0 @@ -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java -index e0d155408d..95f3a96aca 100644 ---- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java -@@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; - import com.fasterxml.jackson.annotation.JsonProperty; - import com.fasterxml.jackson.core.type.TypeReference; - import com.google.common.collect.ImmutableSet; --import org.apache.druid.indexing.common.TaskLock; - import org.apache.druid.indexing.common.task.IndexTaskUtils; - import org.apache.druid.indexing.common.task.Task; - import org.apache.druid.indexing.overlord.CriticalAction; -@@ -31,14 +30,10 @@ import org.apache.druid.indexing.overlord.DataSourceMetadata; - import org.apache.druid.indexing.overlord.SegmentPublishResult; - import org.apache.druid.indexing.overlord.TaskLockInfo; - import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; --import org.apache.druid.query.DruidMetrics; - import org.apache.druid.segment.SegmentUtils; - import org.apache.druid.timeline.DataSegment; --import org.joda.time.Interval; - - import javax.annotation.Nullable; --import java.util.HashMap; --import java.util.HashSet; - import java.util.Map; - import java.util.Set; - import java.util.stream.Collectors; -@@ -111,42 +106,22 @@ public class SegmentTransactionalAppendAction implements TaskAction allSegments = new HashSet<>(segments); -- -- String datasource = task.getDataSource(); -- Map replaceLocks = new HashMap<>(); -- for (TaskLock lock : TaskLocks.findReplaceLocksForSegments(datasource, toolbox.getTaskLockbox(), segments)) { -- replaceLocks.put(lock.getInterval(), lock); -- } -- Map appendSegmentLockMap = new HashMap<>(); -- Set taskLockInfos = new HashSet<>(); -- for (TaskLock taskLock : replaceLocks.values()) { -- taskLockInfos.add(getTaskLockInfo(taskLock)); -- } -- -- for (DataSegment segment : segments) { -- Interval interval = segment.getInterval(); -- for (Interval key : replaceLocks.keySet()) { -- if (key.contains(interval)) { -- appendSegmentLockMap.put(segment, getTaskLockInfo(replaceLocks.get(key))); -- } -- } -- } -+ final String datasource = task.getDataSource(); -+ final Map segmentToReplaceLock -+ = TaskLocks.findReplaceLocksCoveringSegments(datasource, toolbox.getTaskLockbox(), segments); - -+ final SegmentPublishResult retVal; - try { - retVal = toolbox.getTaskLockbox().doInCriticalSection( - task, -- allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), -+ segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), - CriticalAction.builder() - .onValidLocks( - () -> toolbox.getIndexerMetadataStorageCoordinator().commitAppendSegments( - segments, - startMetadata, - endMetadata, -- appendSegmentLockMap, -- taskLockInfos -+ segmentToReplaceLock - ) - ) - .onInvalidLocks( -@@ -168,29 +143,17 @@ public class SegmentTransactionalAppendAction implements TaskActionbuilder() - .onValidLocks( -- () -> toolbox.getIndexerMetadataStorageCoordinator().commitSegments( -+ () -> toolbox.getIndexerMetadataStorageCoordinator().commitSegmentsAndMetadata( - segments, - segmentsToBeDropped, - startMetadata, -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java -index 4dd3ea7a38..255628e262 100644 ---- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java -@@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; - import com.fasterxml.jackson.annotation.JsonProperty; - import com.fasterxml.jackson.core.type.TypeReference; - import com.google.common.collect.ImmutableSet; --import org.apache.druid.indexing.common.TaskLock; - import org.apache.druid.indexing.common.task.IndexTaskUtils; - import org.apache.druid.indexing.common.task.Task; - import org.apache.druid.indexing.overlord.CriticalAction; -@@ -33,10 +32,8 @@ import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; - import org.apache.druid.query.DruidMetrics; - import org.apache.druid.segment.SegmentUtils; - import org.apache.druid.timeline.DataSegment; --import org.joda.time.Interval; - - import javax.annotation.Nullable; --import java.util.HashMap; - import java.util.HashSet; - import java.util.Map; - import java.util.Set; -@@ -119,32 +116,21 @@ public class SegmentTransactionalReplaceAction implements TaskAction allSegments = new HashSet<>(segments); -- -- TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), allSegments); -- - String datasource = task.getDataSource(); -- Map replaceLocks = new HashMap<>(); -- for (TaskLock lock : TaskLocks.findReplaceLocksForSegments(datasource, toolbox.getTaskLockbox(), segments)) { -- replaceLocks.put(lock.getInterval(), lock); -- } -- Set taskLockInfos = new HashSet<>(); -- for (TaskLock taskLock : replaceLocks.values()) { -- taskLockInfos.add(getTaskLockInfo(taskLock)); -- } -+ final Map segmentToReplaceLock -+ = TaskLocks.findReplaceLocksCoveringSegments(datasource, toolbox.getTaskLockbox(), segments); - -+ final SegmentPublishResult retVal; - try { - retVal = toolbox.getTaskLockbox().doInCriticalSection( - task, -- allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), -+ segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()), - CriticalAction.builder() - .onValidLocks( - () -> toolbox.getIndexerMetadataStorageCoordinator().commitReplaceSegments( - segments, - segmentsToBeDropped, -- taskLockInfos -+ new HashSet<>(segmentToReplaceLock.values()) - ) - ) - .onInvalidLocks( -@@ -166,28 +152,20 @@ public class SegmentTransactionalReplaceAction implements TaskAction findReplaceLocksForSegments( -+ /** -+ * Finds locks of type {@link TaskLockType#REPLACE} for each of the given segments -+ * that have an interval completely covering the interval of the respective segments. -+ * -+ * @return Map from segment to REPLACE lock that completely covers it. The map -+ * does not contain an entry for segments that have no covering REPLACE lock. -+ */ -+ public static Map findReplaceLocksCoveringSegments( - final String datasource, - final TaskLockbox taskLockbox, -- final Collection segments -+ final Set segments - ) - { -- final Set found = new HashSet<>(); -- final Set locks = taskLockbox.getAllReplaceLocksForDatasource(datasource); -- segments.forEach(segment -> { -- locks.forEach(lock -> { -- if (lock.getGranularity() == LockGranularity.TIME_CHUNK) { -- final TimeChunkLock timeChunkLock = (TimeChunkLock) lock; -- if (timeChunkLock.getInterval().contains(segment.getInterval()) -- && timeChunkLock.getDataSource().equals(segment.getDataSource())) { -- found.add(lock); -- } -+ // Identify unique segment intervals -+ final Map> intervalToSegments = new HashMap<>(); -+ segments.forEach( -+ segment -> intervalToSegments.computeIfAbsent( -+ segment.getInterval(), interval -> new ArrayList<>() -+ ).add(segment) -+ ); -+ -+ final Set replaceLocks = taskLockbox.getAllReplaceLocksForDatasource(datasource).stream() -+ .map(TaskLocks::toLockInfo) -+ .collect(Collectors.toSet()); -+ -+ final Map segmentToReplaceLock = new HashMap<>(); -+ -+ intervalToSegments.forEach((interval, segmentsForInterval) -> { -+ // For each interval, find the lock that covers it, if any -+ for (TaskLockInfo lock : replaceLocks) { -+ if (lock.getInterval().contains(interval)) { -+ segmentsForInterval.forEach(s -> segmentToReplaceLock.put(s, lock)); -+ return; - } -- }); -+ } - }); -- return found; -+ -+ return segmentToReplaceLock; -+ } -+ -+ public static TaskLockInfo toLockInfo(TaskLock taskLock) -+ { -+ return new TaskLockInfo(taskLock.getInterval(), taskLock.getVersion()); - } - - public static List findLocksForSegments( -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java -index cd7a52f772..20f7584c8e 100644 ---- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java -@@ -33,6 +33,7 @@ import org.apache.druid.server.security.ForbiddenException; - import org.apache.druid.server.security.Resource; - import org.apache.druid.server.security.ResourceAction; - import org.apache.druid.server.security.ResourceType; -+import org.apache.druid.timeline.DataSegment; - import org.apache.druid.utils.CircularBuffer; - import org.joda.time.DateTime; - -@@ -141,4 +142,14 @@ public class IndexTaskUtils - metricBuilder.setDimension(DruidMetrics.TASK_ID, taskStatus.getId()); - metricBuilder.setDimension(DruidMetrics.TASK_STATUS, taskStatus.getStatusCode().toString()); - } -+ -+ public static void setSegmentDimensions( -+ ServiceMetricEvent.Builder metricBuilder, -+ DataSegment segment -+ ) -+ { -+ final String partitionType = segment.getShardSpec() == null ? null : segment.getShardSpec().getType(); -+ metricBuilder.setDimension(DruidMetrics.PARTITIONING_TYPE, partitionType); -+ metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); -+ } - } -diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java -index db94038e07..eaeb74866f 100644 ---- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java -+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java -@@ -1523,14 +1523,14 @@ public class AppenderatorDriverRealtimeIndexTaskTest extends InitializedNullHand - } - - @Override -- public SegmentPublishResult commitSegments( -+ public SegmentPublishResult commitSegmentsAndMetadata( - Set segments, - Set segmentsToDrop, - DataSourceMetadata startMetadata, - DataSourceMetadata endMetadata - ) throws IOException - { -- SegmentPublishResult result = super.commitSegments(segments, segmentsToDrop, startMetadata, endMetadata); -+ SegmentPublishResult result = super.commitSegmentsAndMetadata(segments, segmentsToDrop, startMetadata, endMetadata); - - Assert.assertFalse( - "Segment latch not initialized, did you forget to call expectPublishSegments?", -diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java -index 950983d77b..d22769c1f7 100644 ---- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java -+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java -@@ -29,7 +29,6 @@ import org.apache.druid.data.input.impl.JsonInputFormat; - import org.apache.druid.data.input.impl.LocalInputSource; - import org.apache.druid.indexer.TaskState; - import org.apache.druid.indexing.common.LockGranularity; --import org.apache.druid.indexing.common.TaskLockType; - import org.apache.druid.indexing.common.TaskToolbox; - import org.apache.druid.indexing.common.actions.TaskActionClient; - import org.apache.druid.indexing.common.task.Tasks; -diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java -index e58df94ef8..16c775b01b 100644 ---- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java -+++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java -@@ -173,8 +173,7 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto - Set segments, - DataSourceMetadata oldCommitMetadata, - DataSourceMetadata newCommitMetadata, -- @Nullable Map segmentLockMap, -- @Nullable Set taskLockInfos -+ @Nullable Map segmentLockMap - ) - { - // Don't actually compare metadata, just do it! -@@ -182,7 +181,7 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto - } - - @Override -- public SegmentPublishResult commitSegments( -+ public SegmentPublishResult commitSegmentsAndMetadata( - Set segments, - Set segmentsToDrop, - @Nullable DataSourceMetadata startMetadata, -diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java b/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java -index a6955ece8b..26fc4cc9db 100644 ---- a/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java -+++ b/server/src/main/java/org/apache/druid/indexing/overlord/DataSourceMetadata.java -@@ -26,7 +26,7 @@ import java.util.Set; - - /** - * Commit metadata for a dataSource. Used by -- * {@link IndexerMetadataStorageCoordinator#commitSegments(Set, Set, DataSourceMetadata, DataSourceMetadata)} -+ * {@link IndexerMetadataStorageCoordinator#commitSegmentsAndMetadata(Set, Set, DataSourceMetadata, DataSourceMetadata)} - * to provide metadata transactions for segment inserts. - * - * Two metadata instances can be added together, and any conflicts are resolved in favor of the right-hand side. -diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java -index e5e09269b9..0edadd9605 100644 ---- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java -+++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java -@@ -254,7 +254,7 @@ public interface IndexerMetadataStorageCoordinator - * commit metadata. - * - * If segmentsToDrop is not null and not empty, this insertion will be atomic with a insert-and-drop on inserting -- * {@param segments} and dropping {@param segmentsToDrop} -+ * {@param segments} and dropping {@param segmentsToDrop}. - * - * @param segments set of segments to add, must all be from the same dataSource - * @param segmentsToDrop set of segments to drop, must all be from the same dataSource -@@ -272,7 +272,7 @@ public interface IndexerMetadataStorageCoordinator - * @throws IllegalArgumentException if startMetadata and endMetadata are not either both null or both non-null - * @throws RuntimeException if the state of metadata storage after this call is unknown - */ -- SegmentPublishResult commitSegments( -+ SegmentPublishResult commitSegmentsAndMetadata( - Set segments, - Set segmentsToDrop, - @Nullable DataSourceMetadata startMetadata, -@@ -283,8 +283,7 @@ public interface IndexerMetadataStorageCoordinator - Set segments, - @Nullable DataSourceMetadata startMetadata, - @Nullable DataSourceMetadata endMetadata, -- @Nullable Map segmentLockMap, -- @Nullable Set taskLockInfos -+ @Nullable Map segmentLockMap - ) throws IOException; - - SegmentPublishResult commitReplaceSegments( -diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java -index 5ee437914a..5e26bc33b6 100644 ---- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java -+++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java -@@ -309,7 +309,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor - @Override - public Set commitSegments(final Set segments) throws IOException - { -- final SegmentPublishResult result = commitSegments(segments, null, null, null); -+ final SegmentPublishResult result = commitSegmentsAndMetadata(segments, null, null, null); - - // Metadata transaction cannot fail because we are not trying to do one. - if (!result.isSuccess()) { -@@ -320,7 +320,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor - } - - @Override -- public SegmentPublishResult commitSegments( -+ public SegmentPublishResult commitSegmentsAndMetadata( - final Set segments, - final Set segmentsToDrop, - @Nullable final DataSourceMetadata startMetadata, -@@ -459,41 +459,31 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor - - try { - return connector.retryTransaction( -- new TransactionCallback() -- { -- @Override -- public SegmentPublishResult inTransaction( -- final Handle handle, -- final TransactionStatus transactionStatus -- ) throws Exception -- { -- // Set definitelyNotUpdated back to false upon retrying. -- definitelyNotUpdated.set(false); -- -+ (handle, transactionStatus) -> { -+ // Set definitelyNotUpdated back to false upon retrying. -+ definitelyNotUpdated.set(false); - -- if (segmentsToDrop != null && !segmentsToDrop.isEmpty()) { -- final DataStoreMetadataUpdateResult result = dropSegmentsWithHandle( -- handle, -- segmentsToDrop, -- dataSource -- ); -- if (result.isFailed()) { -- // Metadata store was definitely not updated. -- transactionStatus.setRollbackOnly(); -- definitelyNotUpdated.set(true); -+ if (segmentsToDrop != null && !segmentsToDrop.isEmpty()) { -+ final DataStoreMetadataUpdateResult result = dropSegmentsWithHandle( -+ handle, -+ segmentsToDrop, -+ dataSource -+ ); -+ if (result.isFailed()) { -+ // Metadata store was definitely not updated. -+ transactionStatus.setRollbackOnly(); -+ definitelyNotUpdated.set(true); - -- if (result.canRetry()) { -- throw new RetryTransactionException(result.getErrorMsg()); -- } else { -- throw new RuntimeException(result.getErrorMsg()); -- } -+ if (result.canRetry()) { -+ throw new RetryTransactionException(result.getErrorMsg()); -+ } else { -+ throw new RuntimeException(result.getErrorMsg()); - } - } -- -- final Set inserted = commitReplaceSegmentBatch(handle, newSegments, usedSegments, taskLockInfos); -- -- return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); - } -+ -+ final Set inserted = commitReplaceSegmentBatch(handle, newSegments, usedSegments, taskLockInfos); -+ return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); - }, - 3, - getSqlMetadataMaxRetry() -@@ -514,43 +504,28 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor - final Set segments, - @Nullable final DataSourceMetadata startMetadata, - @Nullable DataSourceMetadata endMetadata, -- @Nullable Map segmentLockMap, -- @Nullable Set taskLockInfos -+ @Nullable Map segmentLockMap - ) - { - if (segments.isEmpty()) { -- throw new IllegalArgumentException("segment set must not be empty"); -+ throw new IllegalArgumentException("No segments to append"); - } - - final String dataSource = segments.iterator().next().getDataSource(); - for (DataSegment segment : segments) { - if (!dataSource.equals(segment.getDataSource())) { -- throw new IllegalArgumentException("segments must all be from the same dataSource"); -+ throw new IllegalArgumentException("All segments to append must belong to the same dataSource"); - } - } - - if ((startMetadata == null && endMetadata != null) || (startMetadata != null && endMetadata == null)) { -- throw new IllegalArgumentException("start/end metadata pair must be either null or non-null"); -+ throw new IllegalArgumentException("Start and end metadata must either be both null or both non-null"); - } - - // Find which segments are used (i.e. not overshadowed). - Set newSegments = new HashSet<>(segments); - final Map> segmentToNewMetadataMap = connector.retryTransaction( -- new TransactionCallback>>() -- { -- @Override -- public Map> inTransaction( -- final Handle handle, -- final TransactionStatus transactionStatus -- ) throws Exception -- { -- return allocateNewSegmentIds( -- handle, -- dataSource, -- segments -- ); -- } -- }, -+ (handle, transactionStatus) -> allocateNewSegmentIds(handle, dataSource, segments), - 0, - SQLMetadataConnector.DEFAULT_MAX_TRIES - ); -@@ -572,49 +547,36 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor - } - Set usedSegments = new HashSet<>(newSegments); - -- -- - final AtomicBoolean definitelyNotUpdated = new AtomicBoolean(false); -- - try { - return connector.retryTransaction( -- new TransactionCallback() -- { -- @Override -- public SegmentPublishResult inTransaction( -- final Handle handle, -- final TransactionStatus transactionStatus -- ) throws Exception -- { -- // Set definitelyNotUpdated back to false upon retrying. -- definitelyNotUpdated.set(false); -+ (handle, transactionStatus) -> { -+ // Set definitelyNotUpdated back to false upon retrying. -+ definitelyNotUpdated.set(false); - -- if (startMetadata != null) { -- final DataStoreMetadataUpdateResult result = updateDataSourceMetadataWithHandle( -- handle, -- dataSource, -- startMetadata, -- endMetadata -- ); -+ if (startMetadata != null) { -+ final DataStoreMetadataUpdateResult result = updateDataSourceMetadataWithHandle( -+ handle, -+ dataSource, -+ startMetadata, -+ endMetadata -+ ); - -- if (result.isFailed()) { -- // Metadata was definitely not updated. -- transactionStatus.setRollbackOnly(); -- definitelyNotUpdated.set(true); -+ if (result.isFailed()) { -+ // Metadata was definitely not updated. -+ transactionStatus.setRollbackOnly(); -+ definitelyNotUpdated.set(true); - -- if (result.canRetry()) { -- throw new RetryTransactionException(result.getErrorMsg()); -- } else { -- throw new RuntimeException(result.getErrorMsg()); -- } -+ if (result.canRetry()) { -+ throw new RetryTransactionException(result.getErrorMsg()); -+ } else { -+ throw new RuntimeException(result.getErrorMsg()); - } - } -- -- -- final Set inserted = commitAppendSegmentBatch(handle, newSegments, usedSegments, segmentLockMap); -- -- return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); - } -+ -+ final Set inserted = commitAppendSegmentBatch(handle, newSegments, usedSegments, segmentLockMap); -+ return SegmentPublishResult.ok(ImmutableSet.copyOf(inserted)); - }, - 3, - getSqlMetadataMaxRetry() -@@ -1299,7 +1261,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor - } - for (DataSegment segment : intervalToSegments.get(interval)) { - SegmentCreateRequest request = new SegmentCreateRequest( -- segment.getId().toString() + version, -+ segment.getId() + version, - null, - version, - NumberedPartialShardSpec.instance() -@@ -1867,7 +1829,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor - for (Map.Entry entry : batch) { - batchMap.put(entry.getKey(), entry.getValue()); - } -- List oldSegments = retrieveSegments(handle, batchMap.keySet()); -+ List oldSegments = retrieveSegmentsById(handle, batchMap.keySet()); - for (DataSegment oldSegment : oldSegments) { - Interval newInterval = oldSegment.getInterval(); - for (DataSegment segment : segments) { -@@ -2007,122 +1969,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor - DataSegment segment = entry.getKey(); - TaskLockInfo lock = entry.getValue(); - appendBatch.add() -- .bind("id", segment.getId().toString() + ":" + lock.hashCode()) -- .bind("dataSource", segment.getDataSource()) -- .bind("start", lock.getInterval().getStartMillis()) -- .bind("end", lock.getInterval().getEndMillis()) -- .bind("segment_id", segment.getId().toString()) -- .bind("lock_version", lock.getVersion()); -- } -- final int[] affectedAppendRows = appendBatch.execute(); -- final boolean succeeded = Arrays.stream(affectedAppendRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); -- if (!succeeded) { -- final List failedToForward = IntStream.range(0, partition.size()) -- .filter(i -> affectedAppendRows[i] != 1) -- .mapToObj(partition::get) -- .map(x -> x.getKey()) -- .collect(Collectors.toList()); -- throw new ISE( -- "Failed to forward appended segments to DB: %s", -- SegmentUtils.commaSeparatedIdentifiers(failedToForward) -- ); -- } -- } -- } -- catch (Exception e) { -- log.errorSegments(segments, "Exception inserting segment metadata"); -- throw e; -- } -- -- return toInsertSegments; -- } -- -- private Set commitSegmentBatch( -- final Handle handle, -- final Set segments, -- final Set usedSegments, -- @Nullable Map appendSegmentLockMap, -- @Nullable Set replaceLocks, -- boolean append -- ) throws IOException -- { -- final Set toInsertSegments = new HashSet<>(); -- try { -- Set existedSegments = segmentExistsBatch(handle, segments); -- log.info("Found these segments already exist in DB: %s", existedSegments); -- for (DataSegment segment : segments) { -- if (!existedSegments.contains(segment.getId().toString())) { -- toInsertSegments.add(segment); -- } -- } -- -- // SELECT -> INSERT can fail due to races; callers must be prepared to retry. -- // Avoiding ON DUPLICATE KEY since it's not portable. -- // Avoiding try/catch since it may cause inadvertent transaction-splitting. -- final List> partitionedSegments = Lists.partition( -- new ArrayList<>(toInsertSegments), -- MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE -- ); -- -- PreparedBatch preparedBatch = handle.prepareBatch( -- StringUtils.format( -- "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload) " -- + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", -- dbTables.getSegmentsTable(), -- connector.getQuoteString() -- ) -- ); -- for (List partition : partitionedSegments) { -- for (DataSegment segment : partition) { -- preparedBatch.add() -- .bind("id", segment.getId().toString()) -- .bind("dataSource", segment.getDataSource()) -- .bind("created_date", DateTimes.nowUtc().toString()) -- .bind("start", segment.getInterval().getStart().toString()) -- .bind("end", segment.getInterval().getEnd().toString()) -- .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) -- .bind("version", segment.getVersion()) -- .bind("used", usedSegments.contains(segment)) -- .bind("payload", jsonMapper.writeValueAsBytes(segment)); -- } -- final int[] affectedInsertRows = preparedBatch.execute(); -- -- final boolean succeeded = Arrays.stream(affectedInsertRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); -- if (succeeded) { -- log.infoSegments(partition, "Published segments to DB"); -- } else { -- final List failedToPublish = IntStream.range(0, partition.size()) -- .filter(i -> affectedInsertRows[i] != 1) -- .mapToObj(partition::get) -- .collect(Collectors.toList()); -- throw new ISE( -- "Failed to publish segments to DB: %s", -- SegmentUtils.commaSeparatedIdentifiers(failedToPublish) -- ); -- } -- } -- -- PreparedBatch appendBatch = handle.prepareBatch( -- StringUtils.format( -- "INSERT INTO %1$s (id, dataSource, start, %2$send%2$s, segment_id, lock_version) " -- + "VALUES (:id, :dataSource, :start, :end, :segment_id, :lock_version)", -- dbTables.getSegmentVersionsTable(), -- connector.getQuoteString() -- ) -- ); -- if (appendSegmentLockMap == null) { -- appendSegmentLockMap = new HashMap<>(); -- } -- final List>> appendSegmentPartitions = Lists.partition( -- new ArrayList<>(appendSegmentLockMap.entrySet()), -- MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE -- ); -- for (List> partition : appendSegmentPartitions) { -- for (Map.Entry entry : partition) { -- DataSegment segment = entry.getKey(); -- TaskLockInfo lock = entry.getValue(); -- appendBatch.add() -- .bind("id", segment.getId().toString() + ":" + lock.hashCode()) -+ .bind("id", segment.getId() + ":" + lock.hashCode()) - .bind("dataSource", segment.getDataSource()) - .bind("start", lock.getInterval().getStartMillis()) - .bind("end", lock.getInterval().getEndMillis()) -@@ -2143,74 +1990,6 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor - ); - } - } -- -- Map segmentsToBeForwarded = new HashMap<>(); -- if (!append) { -- segmentsToBeForwarded = getAppendedSegmentIds( -- handle, -- segments.iterator().next().getDataSource(), -- replaceLocks -- ); -- } -- final int numCorePartitions = segments.size(); -- int partitionNum = segments.size(); -- final List>> forwardSegmentsBatch = Lists.partition( -- new ArrayList<>(segmentsToBeForwarded.entrySet()), -- MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE -- ); -- for (List> batch : forwardSegmentsBatch) { -- Map batchMap = new HashMap<>(); -- for (Map.Entry entry : batch) { -- batchMap.put(entry.getKey(), entry.getValue()); -- } -- List oldSegments = retrieveSegments(handle, batchMap.keySet()); -- for (DataSegment oldSegment : oldSegments) { -- Interval newInterval = oldSegment.getInterval(); -- for (DataSegment segment : segments) { -- if (segment.getInterval().overlaps(newInterval)) { -- if (segment.getInterval().contains(newInterval)) { -- newInterval = segment.getInterval(); -- } else { -- throw new ISE("Incompatible segment intervals for commit: [%s] and [%s].", -- newInterval, -- segment.getInterval() -- ); -- } -- } -- } -- TaskLockInfo lock = batchMap.get(oldSegment.getId().toString()); -- ShardSpec shardSpec = new NumberedShardSpec(partitionNum++, numCorePartitions); -- DataSegment newSegment = new DataSegment( -- oldSegment.getDataSource(), -- newInterval, -- lock.getVersion(), -- oldSegment.getLoadSpec(), -- oldSegment.getDimensions(), -- oldSegment.getMetrics(), -- shardSpec, -- oldSegment.getBinaryVersion(), -- oldSegment.getSize() -- ); -- preparedBatch.add() -- .bind("id", newSegment.getId().toString()) -- .bind("dataSource", newSegment.getDataSource()) -- .bind("created_date", DateTimes.nowUtc().toString()) -- .bind("start", newSegment.getInterval().getStart().toString()) -- .bind("end", newSegment.getInterval().getEnd().toString()) -- .bind("partitioned", (newSegment.getShardSpec() instanceof NoneShardSpec) ? false : true) -- .bind("version", newSegment.getVersion()) -- .bind("used", true) -- .bind("payload", jsonMapper.writeValueAsBytes(newSegment)); -- } -- final int[] affectedInsertRows = preparedBatch.execute(); -- -- final boolean succeeded = Arrays.stream(affectedInsertRows).allMatch(eachAffectedRow -> eachAffectedRow == 1); -- if (succeeded) { -- log.info("Published segments with updated metadata to DB"); -- } else { -- throw new ISE("Failed to update segment metadatas in DB"); -- } -- } - } - catch (Exception e) { - log.errorSegments(segments, "Exception inserting segment metadata"); -@@ -2220,37 +1999,22 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor - return toInsertSegments; - } - -- private List retrieveSegments(final Handle handle, final Set segmentIds) -+ private List retrieveSegmentsById(Handle handle, Set segmentIds) - { -- final StringBuilder sb = new StringBuilder(); -- sb.append("SELECT payload FROM %s WHERE id in ("); -- -- List segmentIdList = new ArrayList<>(segmentIds); -- int n = segmentIdList.size(); -- for (int i = 0; i < n; i++) { -- sb.append("'"); -- sb.append(segmentIdList.get(i)); -- sb.append("'"); -- if (i < n - 1) { -- sb.append(", "); -- } -- } -- -- sb.append(")"); -- -- final Query> sql = handle -- .createQuery(StringUtils.format(sb.toString(), dbTables.getSegmentsTable())) -- .setFetchSize(connector.getStreamingFetchSize()); -+ final String segmentIdCsv = segmentIds.stream().map(id -> "'" + id + "'") -+ .collect(Collectors.joining(",")); -+ final Query> query = handle.createQuery( -+ StringUtils.format( -+ "SELECT payload FROM %s WHERE id in (%s)", -+ segmentIdCsv, dbTables.getSegmentsTable() -+ ) -+ ).setFetchSize(connector.getStreamingFetchSize()); - -- final ResultIterator resultIterator = -- sql.map((index, r, ctx) -> JacksonUtils.readValue(jsonMapper, r.getBytes(1), DataSegment.class)) -- .iterator(); -+ ResultIterator resultIterator = query.map( -+ (index, r, ctx) -> JacksonUtils.readValue(jsonMapper, r.getBytes(1), DataSegment.class) -+ ).iterator(); - -- List retVal = new ArrayList<>(); -- while (resultIterator.hasNext()) { -- retVal.add(resultIterator.next()); -- } -- return retVal; -+ return Lists.newArrayList(resultIterator); - } - - private Map getAppendedSegmentIds( -@@ -2383,7 +2147,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor - * - * @return SUCCESS if dataSource metadata was updated from matching startMetadata to matching endMetadata, FAILURE or - * TRY_AGAIN if it definitely was not updated. This guarantee is meant to help -- * {@link #commitSegments(Set, Set, DataSourceMetadata, DataSourceMetadata)} -+ * {@link #commitSegmentsAndMetadata(Set, Set, DataSourceMetadata, DataSourceMetadata)} - * achieve its own guarantee. - * - * @throws RuntimeException if state is unknown after this call -@@ -2513,7 +2277,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor - * - * @return SUCCESS if segment was marked unused, FAILURE or - * TRY_AGAIN if it definitely was not updated. This guarantee is meant to help -- * {@link #commitSegments(Set, Set, DataSourceMetadata, DataSourceMetadata)} -+ * {@link #commitSegmentsAndMetadata(Set, Set, DataSourceMetadata, DataSourceMetadata)} - * achieve its own guarantee. - * - * @throws RuntimeException if state is unknown after this call -diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java -index a802c3c655..669ffefae2 100644 ---- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java -+++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java -@@ -346,7 +346,7 @@ public abstract class SQLMetadataConnector implements MetadataStorageConnector - ); - } - -- public void createSegmentVersionTable(final String tableName) -+ private void createSegmentVersionTable(final String tableName) - { - createTable( - tableName, -diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java -index eb55f04e20..8845a3beef 100644 ---- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java -+++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java -@@ -558,7 +558,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest - public void testTransactionalAnnounceSuccess() throws IOException - { - // Insert first segment. -- final SegmentPublishResult result1 = coordinator.commitSegments( -+ final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( - ImmutableSet.of(defaultSegment), - ImmutableSet.of(), - new ObjectMetadata(null), -@@ -577,7 +577,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest - ); - - // Insert second segment. -- final SegmentPublishResult result2 = coordinator.commitSegments( -+ final SegmentPublishResult result2 = coordinator.commitSegmentsAndMetadata( - ImmutableSet.of(defaultSegment2), - ImmutableSet.of(), - new ObjectMetadata(ImmutableMap.of("foo", "bar")), -@@ -634,7 +634,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest - }; - - // Insert first segment. -- final SegmentPublishResult result1 = failOnceCoordinator.commitSegments( -+ final SegmentPublishResult result1 = failOnceCoordinator.commitSegmentsAndMetadata( - ImmutableSet.of(defaultSegment), - ImmutableSet.of(), - new ObjectMetadata(null), -@@ -656,7 +656,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest - attemptCounter.set(0); - - // Insert second segment. -- final SegmentPublishResult result2 = failOnceCoordinator.commitSegments( -+ final SegmentPublishResult result2 = failOnceCoordinator.commitSegmentsAndMetadata( - ImmutableSet.of(defaultSegment2), - ImmutableSet.of(), - new ObjectMetadata(ImmutableMap.of("foo", "bar")), -@@ -687,7 +687,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest - @Test - public void testTransactionalAnnounceFailDbNullWantNotNull() throws IOException - { -- final SegmentPublishResult result1 = coordinator.commitSegments( -+ final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( - ImmutableSet.of(defaultSegment), - ImmutableSet.of(), - new ObjectMetadata(ImmutableMap.of("foo", "bar")), -@@ -721,7 +721,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest - .build(); - Set dropSegments = ImmutableSet.of(existingSegment1, existingSegment2, dataSegmentBar); - -- final SegmentPublishResult result1 = coordinator.commitSegments( -+ final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( - SEGMENTS, - dropSegments, - null, -@@ -750,7 +750,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest - retrieveUsedSegmentIds() - ); - -- final SegmentPublishResult result1 = coordinator.commitSegments( -+ final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( - SEGMENTS, - ImmutableSet.of(existingSegment1, existingSegment2), - null, -@@ -788,7 +788,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest - ); - - Set dropSegments = ImmutableSet.of(existingSegment1, defaultSegment4); -- final SegmentPublishResult result1 = coordinator.commitSegments( -+ final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( - SEGMENTS, - dropSegments, - null, -@@ -809,7 +809,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest - @Test - public void testTransactionalAnnounceFailDbNotNullWantNull() throws IOException - { -- final SegmentPublishResult result1 = coordinator.commitSegments( -+ final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( - ImmutableSet.of(defaultSegment), - ImmutableSet.of(), - new ObjectMetadata(null), -@@ -817,7 +817,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest - ); - Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(defaultSegment)), result1); - -- final SegmentPublishResult result2 = coordinator.commitSegments( -+ final SegmentPublishResult result2 = coordinator.commitSegmentsAndMetadata( - ImmutableSet.of(defaultSegment2), - ImmutableSet.of(), - new ObjectMetadata(null), -@@ -835,7 +835,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest - @Test - public void testTransactionalAnnounceFailDbNotNullWantDifferent() throws IOException - { -- final SegmentPublishResult result1 = coordinator.commitSegments( -+ final SegmentPublishResult result1 = coordinator.commitSegmentsAndMetadata( - ImmutableSet.of(defaultSegment), - ImmutableSet.of(), - new ObjectMetadata(null), -@@ -843,7 +843,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest - ); - Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(defaultSegment)), result1); - -- final SegmentPublishResult result2 = coordinator.commitSegments( -+ final SegmentPublishResult result2 = coordinator.commitSegmentsAndMetadata( - ImmutableSet.of(defaultSegment2), - ImmutableSet.of(), - new ObjectMetadata(ImmutableMap.of("foo", "qux")), -@@ -1391,7 +1391,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest - @Test - public void testDeleteDataSourceMetadata() throws IOException - { -- coordinator.commitSegments( -+ coordinator.commitSegmentsAndMetadata( - ImmutableSet.of(defaultSegment), - ImmutableSet.of(), - new ObjectMetadata(null), -@@ -2347,7 +2347,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest - @Test - public void testRemoveDataSourceMetadataOlderThanDatasourceActiveShouldNotBeDeleted() throws Exception - { -- coordinator.commitSegments( -+ coordinator.commitSegmentsAndMetadata( - ImmutableSet.of(defaultSegment), - ImmutableSet.of(), - new ObjectMetadata(null), -@@ -2376,7 +2376,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest - @Test - public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveAndOlderThanTimeShouldBeDeleted() throws Exception - { -- coordinator.commitSegments( -+ coordinator.commitSegmentsAndMetadata( - ImmutableSet.of(defaultSegment), - ImmutableSet.of(), - new ObjectMetadata(null), -@@ -2402,7 +2402,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest - public void testRemoveDataSourceMetadataOlderThanDatasourceNotActiveButNotOlderThanTimeShouldNotBeDeleted() - throws Exception - { -- coordinator.commitSegments( -+ coordinator.commitSegmentsAndMetadata( - ImmutableSet.of(defaultSegment), - ImmutableSet.of(), - new ObjectMetadata(null), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java index be4b8d8228c3..ae111bb13d74 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java @@ -31,7 +31,6 @@ import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.overlord.SegmentPublishResult; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; @@ -58,12 +57,12 @@ public ActionsTestTask(String datasource, TaskActionClientFactory factory) public TaskLock acquireReplaceLockOn(Interval interval) { - return tryTimeChunkLock(interval, TaskLockType.REPLACE); + return runAction(new TimeChunkLockTryAcquireAction(TaskLockType.REPLACE, interval)); } public TaskLock acquireAppendLockOn(Interval interval) { - return tryTimeChunkLock(interval, TaskLockType.APPEND); + return runAction(new TimeChunkLockTryAcquireAction(TaskLockType.APPEND, interval)); } public SegmentPublishResult commitReplaceSegments(DataSegment... segments) @@ -98,18 +97,6 @@ public SegmentIdWithShardSpec allocateSegmentForTimestamp(DateTime timestamp, Gr ); } - private TaskLock tryTimeChunkLock(Interval interval, TaskLockType lockType) - { - final TaskLock lock = runAction(new TimeChunkLockTryAcquireAction(lockType, interval)); - if (lock == null) { - throw new ISE("Could not acquire [%s] lock on interval[%s] for task[%s]", lockType, interval, getId()); - } else if (lock.isRevoked()) { - throw new ISE("Acquired [%s] lock on interval[%s] for task[%s] has been revoked.", lockType, interval, getId()); - } - - return lock; - } - private T runAction(TaskAction action) { return execute(() -> client.submit(action)); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAppendTest.java index 81daeaf8feba..c31e3d49145d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAppendTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import org.apache.druid.indexing.common.MultipleFileTaskReportFileWriter; +import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskStorageDirTracker; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolboxFactory; @@ -75,7 +76,7 @@ *

* The tests verify the interleaving of the following actions: *

    - *
  • LOCK: Acquisiting of a lock on an interval by a replace task
  • + *
  • LOCK: Acquisition of a lock on an interval by a replace task
  • *
  • ALLOCATE: Allocation of a pending segment by an append task
  • *
  • REPLACE: Commit of segments created by a replace task
  • *
  • APPEND: Commit of segments created by an append task
  • @@ -305,6 +306,340 @@ public void testAllocateAppendLockReplace() verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); } + @Test + public void testLockReplaceMonthAllocateAppendDay() + { + String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); + + final DataSegment segmentV10 = createSegment(JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + verifyIntervalHasUsedSegments(JAN_23, segmentV10); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); + + // Verify that the allocated segment takes the version and interval of previous replace + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(JAN_23, pendingSegment.getInterval()); + Assert.assertEquals(v1, pendingSegment.getVersion()); + + final DataSegment segmentV11 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV11); + + verifyIntervalHasUsedSegments(JAN_23, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(JAN_23, segmentV10, segmentV11); + } + + @Test + public void testLockAllocateAppendDayReplaceMonth() + { + final String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); + + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV01); + + final DataSegment segmentV10 = createSegment(JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + + // Verify that append segment gets upgraded to replace version + final DataSegment segmentV11 = DataSegment.builder(segmentV01) + .version(v1) + .interval(segmentV10.getInterval()) + .shardSpec(new NumberedShardSpec(1, 1)) + .build(); + verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + } + + @Test + public void testLockAllocateReplaceMonthAppendDay() + { + final String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); + + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + + final DataSegment segmentV10 = createSegment(JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); + + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); + + // Verify that append segment gets upgraded to replace version + final DataSegment segmentV11 = DataSegment.builder(segmentV01) + .version(v1) + .interval(segmentV10.getInterval()) + .shardSpec(new NumberedShardSpec(1, 1)) + .build(); + verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + } + + @Test + public void testAllocateLockReplaceMonthAppendDay() + { + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + + final String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); + + final DataSegment segmentV10 = createSegment(JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); + + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); + + // Verify that append segment gets upgraded to replace version + final DataSegment segmentV11 = DataSegment.builder(segmentV01) + .version(v1) + .interval(segmentV10.getInterval()) + .shardSpec(new NumberedShardSpec(1, 1)) + .build(); + verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + } + + @Test + public void testAllocateLockAppendDayReplaceMonth() + { + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + + final String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); + + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV01); + + final DataSegment segmentV10 = createSegment(JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + + // Verify that append segment gets upgraded to replace version + final DataSegment segmentV11 = DataSegment.builder(segmentV01) + .version(v1) + .interval(segmentV10.getInterval()) + .shardSpec(new NumberedShardSpec(1, 1)) + .build(); + verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + } + + @Test + public void testAllocateAppendDayLockReplaceMonth() + { + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV01); + + final String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); + + final DataSegment segmentV10 = createSegment(JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + + // Verify that the old segment gets completely replaced + verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); + } + + @Test + public void testLockReplaceDayAllocateAppendMonth() + { + final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); + + final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); + + TaskLock appendLock = appendTask.acquireAppendLockOn(JAN_23); + Assert.assertNull(appendLock); + + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + } + + @Test + public void testLockAllocateAppendMonthReplaceDay() + { + final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); + + TaskLock appendLock = appendTask.acquireAppendLockOn(JAN_23); + Assert.assertNull(appendLock); + + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); + Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV01); + + final DataSegment segmentV10 = createSegment(JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + + // Verify that append segment gets upgraded to replace version + final DataSegment segmentV11 = DataSegment.builder(segmentV01) + .version(v1) + .interval(segmentV10.getInterval()) + .shardSpec(new NumberedShardSpec(1, 1)) + .build(); + verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + } + + @Test + public void testLockAllocateReplaceDayAppendMonth() + { + final String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); + + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + + final DataSegment segmentV10 = createSegment(JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); + + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); + + // Verify that append segment gets upgraded to replace version + final DataSegment segmentV11 = DataSegment.builder(segmentV01) + .version(v1) + .interval(segmentV10.getInterval()) + .shardSpec(new NumberedShardSpec(1, 1)) + .build(); + verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + } + + @Test + public void testAllocateLockReplaceDayAppendMonth() + { + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + + final String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); + + final DataSegment segmentV10 = createSegment(JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); + + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); + + // Verify that append segment gets upgraded to replace version + final DataSegment segmentV11 = DataSegment.builder(segmentV01) + .version(v1) + .interval(segmentV10.getInterval()) + .shardSpec(new NumberedShardSpec(1, 1)) + .build(); + verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + } + + @Test + public void testAllocateLockAppendMonthReplaceDay() + { + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + + final String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); + + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV01); + + final DataSegment segmentV10 = createSegment(JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + + // Verify that append segment gets upgraded to replace version + final DataSegment segmentV11 = DataSegment.builder(segmentV01) + .version(v1) + .interval(segmentV10.getInterval()) + .shardSpec(new NumberedShardSpec(1, 1)) + .build(); + verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + } + + @Test + public void testAllocateAppendMonthLockReplaceDay() + { + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV01); + + final String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); + + final DataSegment segmentV10 = createSegment(JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + + // Verify that the old segment gets completely replaced + verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); + } + @Test public void testLockReplaceAllocateLockReplaceLockReplaceAppend() { From ab0b400064fed2073c65cf9ec87fd27c70693fea Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 19 Sep 2023 12:44:36 +0530 Subject: [PATCH 32/43] Remove extra change --- .../main/java/org/apache/druid/indexing/common/task/Task.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index 59a98e2ae7b3..58a2ad435b0f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -79,7 +79,7 @@ @Type(name = "index_realtime", value = RealtimeIndexTask.class), @Type(name = "index_realtime_appenderator", value = AppenderatorDriverRealtimeIndexTask.class), @Type(name = "noop", value = NoopTask.class), - @Type(name = "compact", value = CompactionTask.class), + @Type(name = "compact", value = CompactionTask.class) }) public interface Task { From 8bb5a131b29cbb5729486ca6c00abe8098f937bc Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 20 Sep 2023 09:27:13 +0530 Subject: [PATCH 33/43] Fix tests --- ...va => ConcurrentReplaceAndAppendTest.java} | 184 ++++++++---------- 1 file changed, 84 insertions(+), 100 deletions(-) rename indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/{ConcurrentReplaceAppendTest.java => ConcurrentReplaceAndAppendTest.java} (83%) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java similarity index 83% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAppendTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java index c31e3d49145d..fd3e4b9a1839 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.common.task.concurrent; +import com.amazonaws.util.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import org.apache.druid.indexing.common.MultipleFileTaskReportFileWriter; @@ -46,6 +47,7 @@ import org.apache.druid.indexing.worker.config.WorkerConfig; 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.granularity.Granularities; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.column.ColumnConfig; @@ -82,7 +84,7 @@ *
  • APPEND: Commit of segments created by an append task
  • *
*/ -public class ConcurrentReplaceAppendTest extends IngestionTestBase +public class ConcurrentReplaceAndAppendTest extends IngestionTestBase { /** * The version used by append jobs when no previous replace job has run on an interval. @@ -170,7 +172,7 @@ public void testLockReplaceAllocateAppend() } @Test - public void testLockAllocateAppendReplace() + public void testLockAllocateAppendDayReplaceDay() { final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); @@ -198,7 +200,7 @@ public void testLockAllocateAppendReplace() } @Test - public void testLockAllocateReplaceAppend() + public void testLockAllocateReplaceDayAppendDay() { final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); @@ -226,7 +228,7 @@ public void testLockAllocateReplaceAppend() } @Test - public void testAllocateLockReplaceAppend() + public void testAllocateLockReplaceDayAppendDay() { appendTask.acquireAppendLockOn(FIRST_OF_JAN_23).getVersion(); final SegmentIdWithShardSpec pendingSegment @@ -254,7 +256,7 @@ public void testAllocateLockReplaceAppend() } @Test - public void testAllocateLockAppendReplace() + public void testAllocateLockAppendDayReplaceDay() { appendTask.acquireAppendLockOn(FIRST_OF_JAN_23).getVersion(); final SegmentIdWithShardSpec pendingSegment @@ -282,7 +284,7 @@ public void testAllocateLockAppendReplace() } @Test - public void testAllocateAppendLockReplace() + public void testAllocateAppendDayLockReplaceDay() { appendTask.acquireAppendLockOn(FIRST_OF_JAN_23).getVersion(); final SegmentIdWithShardSpec pendingSegment @@ -357,7 +359,7 @@ public void testLockAllocateAppendDayReplaceMonth() .shardSpec(new NumberedShardSpec(1, 1)) .build(); verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10, segmentV11); - verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(JAN_23, segmentV10, segmentV11); } @Test @@ -387,7 +389,7 @@ public void testLockAllocateReplaceMonthAppendDay() .shardSpec(new NumberedShardSpec(1, 1)) .build(); verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10, segmentV11); - verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(JAN_23, segmentV10, segmentV11); } @Test @@ -417,7 +419,7 @@ public void testAllocateLockReplaceMonthAppendDay() .shardSpec(new NumberedShardSpec(1, 1)) .build(); verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10, segmentV11); - verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(JAN_23, segmentV10, segmentV11); } @Test @@ -447,7 +449,7 @@ public void testAllocateLockAppendDayReplaceMonth() .shardSpec(new NumberedShardSpec(1, 1)) .build(); verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10, segmentV11); - verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(JAN_23, segmentV10, segmentV11); } @Test @@ -472,7 +474,7 @@ public void testAllocateAppendDayLockReplaceMonth() // Verify that the old segment gets completely replaced verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10); - verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); + verifyIntervalHasVisibleSegments(JAN_23, segmentV10); } @Test @@ -485,159 +487,141 @@ public void testLockReplaceDayAllocateAppendMonth() verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); + // Verify that an APPEND lock cannot be acquired on month TaskLock appendLock = appendTask.acquireAppendLockOn(JAN_23); Assert.assertNull(appendLock); + // Verify that new segment gets allocated with DAY granularity even though preferred was MONTH final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); - Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + Assert.assertEquals(v1, pendingSegment.getVersion()); + Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); + + final DataSegment segmentV11 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV11); + + verifyIntervalHasUsedSegments(JAN_23, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(JAN_23, segmentV10, segmentV11); } @Test - public void testLockAllocateAppendMonthReplaceDay() + public void negativeTestLockAllocateAppendMonthReplaceDay_appendedSegmentOvershadowsReplace() { final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); + // Verify that an APPEND lock cannot be acquired on month TaskLock appendLock = appendTask.acquireAppendLockOn(JAN_23); Assert.assertNull(appendLock); + // Verify that the segment is allocated for DAY granularity but version is neither v0 nor v1 final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); + final String vAppend = pendingSegment.getVersion(); Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); - Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); + Assert.assertTrue(vAppend.compareTo(SEGMENT_V0) > 0); + Assert.assertTrue(vAppend.compareTo(v1) > 0); - final DataSegment segmentV01 = asSegment(pendingSegment); - appendTask.commitAppendSegments(segmentV01); + final DataSegment appendedSegment = asSegment(pendingSegment); + appendTask.commitAppendSegments(appendedSegment); - verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01); - verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV01); + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, appendedSegment); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, appendedSegment); - final DataSegment segmentV10 = createSegment(JAN_23, v1); + final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); replaceTask.commitReplaceSegments(segmentV10); // Verify that append segment gets upgraded to replace version - final DataSegment segmentV11 = DataSegment.builder(segmentV01) + final DataSegment segmentV11 = DataSegment.builder(appendedSegment) .version(v1) .interval(segmentV10.getInterval()) .shardSpec(new NumberedShardSpec(1, 1)) .build(); - verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10, segmentV11); - verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + verifyIntervalHasUsedSegments(JAN_23, appendedSegment, segmentV10, segmentV11); + + // Fix this: Appended segment incorrectly overshadows replace segment + verifyIntervalHasVisibleSegments(JAN_23, appendedSegment); } @Test - public void testLockAllocateReplaceDayAppendMonth() + public void negativeTestLockAllocateReplaceDayAppendMonth_appendedSegmentOvershadowsReplace() { - final String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); - - appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); - final SegmentIdWithShardSpec pendingSegment - = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); - Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); - Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); - - final DataSegment segmentV10 = createSegment(JAN_23, v1); - replaceTask.commitReplaceSegments(segmentV10); - - verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); - verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); - - final DataSegment segmentV01 = asSegment(pendingSegment); - appendTask.commitAppendSegments(segmentV01); + final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); - // Verify that append segment gets upgraded to replace version - final DataSegment segmentV11 = DataSegment.builder(segmentV01) - .version(v1) - .interval(segmentV10.getInterval()) - .shardSpec(new NumberedShardSpec(1, 1)) - .build(); - verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10, segmentV11); - verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); - } + // Verify that an APPEND lock cannot be acquired on month + TaskLock appendLock = appendTask.acquireAppendLockOn(JAN_23); + Assert.assertNull(appendLock); - @Test - public void testAllocateLockReplaceDayAppendMonth() - { - appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); + // Verify that the segment is allocated for DAY granularity but version is neither v0 nor v1 final SegmentIdWithShardSpec pendingSegment - = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); + final String vAppend = pendingSegment.getVersion(); Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); - Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); - - final String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); + Assert.assertTrue(vAppend.compareTo(SEGMENT_V0) > 0); + Assert.assertTrue(vAppend.compareTo(v1) > 0); - final DataSegment segmentV10 = createSegment(JAN_23, v1); + final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); replaceTask.commitReplaceSegments(segmentV10); verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); - final DataSegment segmentV01 = asSegment(pendingSegment); - appendTask.commitAppendSegments(segmentV01); + final DataSegment appendedSegment = asSegment(pendingSegment); + appendTask.commitAppendSegments(appendedSegment); - // Verify that append segment gets upgraded to replace version - final DataSegment segmentV11 = DataSegment.builder(segmentV01) - .version(v1) - .interval(segmentV10.getInterval()) - .shardSpec(new NumberedShardSpec(1, 1)) - .build(); - verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10, segmentV11); - verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + // Fix this: Appended segment incorrectly overshadows replace segment + verifyIntervalHasUsedSegments(JAN_23, appendedSegment, segmentV10); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, appendedSegment); } @Test - public void testAllocateLockAppendMonthReplaceDay() + public void testAllocateLockReplaceDayAppendMonth() { - appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); + appendTask.acquireAppendLockOn(JAN_23); final SegmentIdWithShardSpec pendingSegment - = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); - Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); + = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); + Assert.assertEquals(JAN_23, pendingSegment.getInterval()); Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); - final String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); + // Verify that replace lock cannot be acquired on MONTH + TaskLock replaceLock = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23); + Assert.assertNull(replaceLock); + + // Verify that segment cannot be committed since there is no lock + final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, SEGMENT_V0); + final ISE exception = Assert.assertThrows(ISE.class, () -> replaceTask.commitReplaceSegments(segmentV10)); + final Throwable throwable = Throwables.getRootCause(exception); + Assert.assertEquals( + StringUtils.format( + "Segments[[%s]] are not covered by locks[[]] for task[%s]", + segmentV10, replaceTask.getId() + ), + throwable.getMessage() + ); final DataSegment segmentV01 = asSegment(pendingSegment); appendTask.commitAppendSegments(segmentV01); - - verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01); - verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV01); - - final DataSegment segmentV10 = createSegment(JAN_23, v1); - replaceTask.commitReplaceSegments(segmentV10); - - // Verify that append segment gets upgraded to replace version - final DataSegment segmentV11 = DataSegment.builder(segmentV01) - .version(v1) - .interval(segmentV10.getInterval()) - .shardSpec(new NumberedShardSpec(1, 1)) - .build(); - verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10, segmentV11); - verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); + verifyIntervalHasUsedSegments(JAN_23, segmentV01); + verifyIntervalHasVisibleSegments(JAN_23, segmentV01); } @Test public void testAllocateAppendMonthLockReplaceDay() { - appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); + appendTask.acquireAppendLockOn(JAN_23); final SegmentIdWithShardSpec pendingSegment - = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); - Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); + = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); + Assert.assertEquals(JAN_23, pendingSegment.getInterval()); Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); final DataSegment segmentV01 = asSegment(pendingSegment); appendTask.commitAppendSegments(segmentV01); - verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01); - verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV01); - - final String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); - - final DataSegment segmentV10 = createSegment(JAN_23, v1); - replaceTask.commitReplaceSegments(segmentV10); + verifyIntervalHasUsedSegments(JAN_23, segmentV01); + verifyIntervalHasVisibleSegments(JAN_23, segmentV01); - // Verify that the old segment gets completely replaced - verifyIntervalHasUsedSegments(JAN_23, segmentV01, segmentV10); - verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); + // Verify that replace lock cannot be acquired on DAY as MONTH is already locked + final TaskLock replaceLock = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23); + Assert.assertNull(replaceLock); } @Test From 9c7d5b2cd548d5d494e25b65976754f01b8862ac Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 20 Sep 2023 10:15:37 +0530 Subject: [PATCH 34/43] Remove unused dependency --- .../common/task/concurrent/ConcurrentReplaceAndAppendTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java index fd3e4b9a1839..4bc23f666541 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java @@ -19,7 +19,7 @@ package org.apache.druid.indexing.common.task.concurrent; -import com.amazonaws.util.Throwables; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import org.apache.druid.indexing.common.MultipleFileTaskReportFileWriter; From 17ab8448a3c9385e03deba619edaf15718ebcda1 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 20 Sep 2023 19:18:44 +0530 Subject: [PATCH 35/43] Add more tests --- .../SegmentTransactionalAppendAction.java | 4 +- .../task/concurrent/ActionsTestTask.java | 4 +- .../ConcurrentReplaceAndAppendTest.java | 114 +++++++++++++++++- .../IndexerSQLMetadataStorageCoordinator.java | 9 +- 4 files changed, 116 insertions(+), 15 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java index 9b90ce61f2b2..e6c1261d261a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java @@ -43,9 +43,7 @@ public class SegmentTransactionalAppendAction implements TaskAction segments; - public static SegmentTransactionalAppendAction create( - Set segments - ) + public static SegmentTransactionalAppendAction create(Set segments) { return new SegmentTransactionalAppendAction(segments); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java index ae111bb13d74..b78efcbc3469 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ActionsTestTask.java @@ -49,9 +49,9 @@ public class ActionsTestTask extends CommandQueueTask private final TaskActionClient client; private final AtomicInteger sequenceId = new AtomicInteger(0); - public ActionsTestTask(String datasource, TaskActionClientFactory factory) + public ActionsTestTask(String datasource, String groupId, TaskActionClientFactory factory) { - super(datasource, null); + super(datasource, groupId); this.client = factory.create(this); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java index 4bc23f666541..78a71d866853 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java @@ -71,6 +71,7 @@ import java.util.Collections; import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; /** * Contains tests to verify behaviour of concurrently running REPLACE and APPEND @@ -93,6 +94,8 @@ public class ConcurrentReplaceAndAppendTest extends IngestionTestBase private static final Interval YEAR_23 = Intervals.of("2023/2024"); private static final Interval JAN_23 = Intervals.of("2023-01/2023-02"); + private static final Interval DEC_23 = Intervals.of("2023-12/2024-01"); + private static final Interval OCT_NOV_DEC_23 = Intervals.of("2023-10-01/2024-01-01"); private static final Interval FIRST_OF_JAN_23 = Intervals.of("2023-01-01/2023-01-02"); private static final String WIKI = "wiki"; @@ -105,6 +108,8 @@ public class ConcurrentReplaceAndAppendTest extends IngestionTestBase private ActionsTestTask appendTask; private ActionsTestTask replaceTask; + private final AtomicInteger groupId = new AtomicInteger(0); + @Before public void setup() { @@ -137,6 +142,7 @@ public void setup() runningTasks.clear(); taskQueue.start(); + groupId.set(0); appendTask = createAndStartTask(); replaceTask = createAndStartTask(); } @@ -156,7 +162,6 @@ public void testLockReplaceAllocateAppend() final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); replaceTask.commitReplaceSegments(segmentV10); - replaceTask.finishRunAndGetStatus(); verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); @@ -189,7 +194,6 @@ public void testLockAllocateAppendDayReplaceDay() final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); replaceTask.commitReplaceSegments(segmentV10); - replaceTask.finishRunAndGetStatus(); // Verify that the segment appended to v0 gets upgraded to v1 final DataSegment segmentV11 = DataSegment.builder(segmentV01) @@ -211,7 +215,6 @@ public void testLockAllocateReplaceDayAppendDay() final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); replaceTask.commitReplaceSegments(segmentV10); - replaceTask.finishRunAndGetStatus(); verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); @@ -239,7 +242,6 @@ public void testAllocateLockReplaceDayAppendDay() final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); replaceTask.commitReplaceSegments(segmentV10); - replaceTask.finishRunAndGetStatus(); verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); @@ -301,7 +303,6 @@ public void testAllocateAppendDayLockReplaceDay() final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); replaceTask.commitReplaceSegments(segmentV10); - replaceTask.finishRunAndGetStatus(); // Verify that the segment appended to v0 gets fully overshadowed verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01, segmentV10); @@ -683,6 +684,107 @@ public void testLockReplaceAllocateLockReplaceLockReplaceAppend() verifyIntervalHasVisibleSegments(YEAR_23, segmentV30, segmentV31); } + @Test + public void testLockReplaceMultipleAppends() + { + final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); + + final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); + + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); + final SegmentIdWithShardSpec pendingSegment + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(segmentV10.getVersion(), pendingSegment.getVersion()); + + final ActionsTestTask appendTask2 = createAndStartTask(); + appendTask2.acquireAppendLockOn(FIRST_OF_JAN_23); + final SegmentIdWithShardSpec pendingSegment2 + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(segmentV10.getVersion(), pendingSegment2.getVersion()); + + final DataSegment segmentV11 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV11); + + final DataSegment segmentV12 = asSegment(pendingSegment2); + appendTask.commitAppendSegments(segmentV12); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10, segmentV11, segmentV12); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11, segmentV12); + } + + @Test + public void testMultipleGranularities() + { + // Allocate segment for Jan 1st + appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); + final SegmentIdWithShardSpec pendingSegment01 + = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); + Assert.assertEquals(SEGMENT_V0, pendingSegment01.getVersion()); + Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment01.getInterval()); + + // Allocate segment for Oct-Dec + final ActionsTestTask appendTask2 = createAndStartTask(); + appendTask2.acquireAppendLockOn(OCT_NOV_DEC_23); + final SegmentIdWithShardSpec pendingSegment02 + = appendTask2.allocateSegmentForTimestamp(OCT_NOV_DEC_23.getStart(), Granularities.QUARTER); + Assert.assertEquals(SEGMENT_V0, pendingSegment02.getVersion()); + Assert.assertEquals(OCT_NOV_DEC_23, pendingSegment02.getInterval()); + + // Append segment for Oct-Dec + final DataSegment segmentV02 = asSegment(pendingSegment02); + appendTask2.commitAppendSegments(segmentV02); + verifyIntervalHasUsedSegments(YEAR_23, segmentV02); + verifyIntervalHasVisibleSegments(YEAR_23, segmentV02); + + // Try to Allocate segment for Dec + final ActionsTestTask appendTask3 = createAndStartTask(); + appendTask3.acquireAppendLockOn(DEC_23); + final SegmentIdWithShardSpec pendingSegment03 + = appendTask3.allocateSegmentForTimestamp(DEC_23.getStart(), Granularities.MONTH); + + // Verify that segment gets allocated for quarter instead of month + Assert.assertEquals(SEGMENT_V0, pendingSegment03.getVersion()); + Assert.assertEquals(OCT_NOV_DEC_23, pendingSegment03.getInterval()); + + // Acquire replace lock on whole year + final String v1 = replaceTask.acquireReplaceLockOn(YEAR_23).getVersion(); + + // Append segment for Jan 1st + final DataSegment segmentV01 = asSegment(pendingSegment01); + appendTask.commitAppendSegments(segmentV01); + verifyIntervalHasUsedSegments(YEAR_23, segmentV01, segmentV02); + verifyIntervalHasVisibleSegments(YEAR_23, segmentV01, segmentV02); + + // Replace segment for whole year + final DataSegment segmentV10 = createSegment(YEAR_23, v1); + replaceTask.commitReplaceSegments(segmentV10); + + final DataSegment segmentV11 = DataSegment.builder(segmentV01) + .version(v1) + .interval(YEAR_23) + .shardSpec(new NumberedShardSpec(1, 1)) + .build(); + + // Verify that segmentV01 is upgraded to segmentV11 and segmentV02 is replaced + verifyIntervalHasUsedSegments(YEAR_23, segmentV01, segmentV02, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(YEAR_23, segmentV10, segmentV11); + + // Append segment for quarter + final DataSegment segmentV03 = asSegment(pendingSegment03); + appendTask3.commitAppendSegments(segmentV03); + + final DataSegment segmentV13 = DataSegment.builder(segmentV03) + .version(v1) + .interval(YEAR_23) + .shardSpec(new NumberedShardSpec(2, 1)) + .build(); + + verifyIntervalHasUsedSegments(YEAR_23, segmentV01, segmentV02, segmentV03, segmentV10, segmentV11, segmentV13); + verifyIntervalHasVisibleSegments(YEAR_23, segmentV10, segmentV11, segmentV13); + } + private static DataSegment asSegment(SegmentIdWithShardSpec pendingSegment) { final SegmentId id = pendingSegment.asSegmentId(); @@ -757,7 +859,7 @@ private DataSegment createSegment(Interval interval, String version) private ActionsTestTask createAndStartTask() { - ActionsTestTask task = new ActionsTestTask(WIKI, taskActionClientFactory); + ActionsTestTask task = new ActionsTestTask(WIKI, "test_" + groupId.incrementAndGet(), taskActionClientFactory); taskQueue.add(task); runningTasks.add(task); return task; diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index b2d4ab22a0e6..9908f9505a3a 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1092,19 +1092,20 @@ Map> getSegmentsToUpgradeOnAppend( Segments.INCLUDING_OVERSHADOWED ); - final Set committedVersions = new HashSet<>(); + final Map> committedVersionToIntervals = new HashMap<>(); final Map> committedIntervalToSegments = new HashMap<>(); for (DataSegment segment : overlappingSegments) { - committedVersions.add(segment.getVersion()); + committedVersionToIntervals.computeIfAbsent(segment.getVersion(), v -> new HashSet<>()) + .add(segment.getInterval()); committedIntervalToSegments.computeIfAbsent(segment.getInterval(), i -> new HashSet<>()) .add(segment); } final Map> appendSegmentToNewIds = new HashMap<>(); - for (String upgradeVersion : committedVersions) { + for (String upgradeVersion : committedVersionToIntervals.keySet()) { Map> segmentsToUpgrade = getSegmentsWithVersionLowerThan( upgradeVersion, - committedIntervalToSegments.keySet(), + committedVersionToIntervals.get(upgradeVersion), appendVersionToSegments ); for (Map.Entry> entry : segmentsToUpgrade.entrySet()) { From e2b04d46ce82c22bf881bc64927962c5bce01c8b Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 21 Sep 2023 13:47:54 +0530 Subject: [PATCH 36/43] Use correct init version for APPEND locks --- .../indexing/common/actions/TaskLocks.java | 8 +++ .../overlord/LockRequestForNewSegment.java | 3 +- .../overlord/TimeChunkLockRequest.java | 9 ++-- .../common/task/IngestionTestBase.java | 1 + .../ConcurrentReplaceAndAppendTest.java | 50 +++++++++---------- 5 files changed, 39 insertions(+), 32 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java index bf16e2369d1e..045b60495f02 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java @@ -28,6 +28,7 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.ReplaceTaskLock; import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; @@ -83,6 +84,13 @@ static boolean isLockCoversSegments( return isLockCoversSegments(taskLockMap, segments); } + public static String defaultLockVersion(TaskLockType lockType) + { + return lockType == TaskLockType.APPEND + ? DateTimes.EPOCH.toString() + : DateTimes.nowUtc().toString(); + } + public static boolean isLockCoversSegments( NavigableMap> taskLockMap, Collection segments diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java index 9d1ecb52b903..c515bfac0396 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java @@ -23,6 +23,7 @@ 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.actions.TaskLocks; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.timeline.partition.PartialShardSpec; @@ -142,7 +143,7 @@ public PartialShardSpec getPartialShardSpec() public String getVersion() { if (version == null) { - version = DateTimes.nowUtc().toString(); + version = TaskLocks.defaultLockVersion(lockType); } return version; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java index e9871da543ba..d92041c7ab04 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java @@ -23,15 +23,14 @@ import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TimeChunkLock; +import org.apache.druid.indexing.common.actions.TaskLocks; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.java.util.common.DateTimes; import org.joda.time.Interval; import javax.annotation.Nullable; public class TimeChunkLockRequest implements LockRequest { - private static final String MIN_VERSION = "1970-01-01T00:00:00.000Z"; private final TaskLockType lockType; private final String groupId; private final String dataSource; @@ -78,10 +77,8 @@ public TimeChunkLockRequest( this.groupId = groupId; this.dataSource = dataSource; this.interval = interval; - if (lockType.equals(TaskLockType.APPEND) && preferredVersion == null) { - this.preferredVersion = MIN_VERSION; - } else if (preferredVersion == null) { - this.preferredVersion = DateTimes.nowUtc().toString(); + if (preferredVersion == null) { + this.preferredVersion = TaskLocks.defaultLockVersion(lockType); } else { this.preferredVersion = preferredVersion; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index d8d1e802b9a4..bec9bd135e74 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -121,6 +121,7 @@ public void setUpIngestionTestBase() throws IOException { EmittingLogger.registerEmitter(new NoopServiceEmitter()); temporaryFolder.create(); + baseDir = temporaryFolder.newFolder(); final SQLMetadataConnector connector = derbyConnectorRule.getConnector(); connector.createTaskTables(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java index 78a71d866853..b78c7af2d321 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java @@ -45,6 +45,7 @@ import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.indexing.overlord.config.TaskQueueConfig; import org.apache.druid.indexing.worker.config.WorkerConfig; +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; @@ -90,7 +91,7 @@ public class ConcurrentReplaceAndAppendTest extends IngestionTestBase /** * The version used by append jobs when no previous replace job has run on an interval. */ - private static final String SEGMENT_V0 = "1970-01-01T00:00:00.000Z"; + private static final String SEGMENT_V0 = DateTimes.EPOCH.toString(); private static final Interval YEAR_23 = Intervals.of("2023/2024"); private static final Interval JAN_23 = Intervals.of("2023-01/2023-02"); @@ -506,7 +507,7 @@ public void testLockReplaceDayAllocateAppendMonth() } @Test - public void negativeTestLockAllocateAppendMonthReplaceDay_appendedSegmentOvershadowsReplace() + public void testLockAllocateAppendMonthReplaceDay() { final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); @@ -514,37 +515,33 @@ public void negativeTestLockAllocateAppendMonthReplaceDay_appendedSegmentOversha TaskLock appendLock = appendTask.acquireAppendLockOn(JAN_23); Assert.assertNull(appendLock); - // Verify that the segment is allocated for DAY granularity but version is neither v0 nor v1 + // Verify that the segment is allocated for DAY granularity final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); - final String vAppend = pendingSegment.getVersion(); Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); - Assert.assertTrue(vAppend.compareTo(SEGMENT_V0) > 0); - Assert.assertTrue(vAppend.compareTo(v1) > 0); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); - final DataSegment appendedSegment = asSegment(pendingSegment); - appendTask.commitAppendSegments(appendedSegment); + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); - verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, appendedSegment); - verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, appendedSegment); + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV01); final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); replaceTask.commitReplaceSegments(segmentV10); // Verify that append segment gets upgraded to replace version - final DataSegment segmentV11 = DataSegment.builder(appendedSegment) + final DataSegment segmentV11 = DataSegment.builder(segmentV01) .version(v1) .interval(segmentV10.getInterval()) .shardSpec(new NumberedShardSpec(1, 1)) .build(); - verifyIntervalHasUsedSegments(JAN_23, appendedSegment, segmentV10, segmentV11); - - // Fix this: Appended segment incorrectly overshadows replace segment - verifyIntervalHasVisibleSegments(JAN_23, appendedSegment); + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); } @Test - public void negativeTestLockAllocateReplaceDayAppendMonth_appendedSegmentOvershadowsReplace() + public void testLockAllocateReplaceDayAppendMonth() { final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); @@ -552,13 +549,11 @@ public void negativeTestLockAllocateReplaceDayAppendMonth_appendedSegmentOversha TaskLock appendLock = appendTask.acquireAppendLockOn(JAN_23); Assert.assertNull(appendLock); - // Verify that the segment is allocated for DAY granularity but version is neither v0 nor v1 + // Verify that the segment is allocated for DAY granularity instead of MONTH final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); - final String vAppend = pendingSegment.getVersion(); Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); - Assert.assertTrue(vAppend.compareTo(SEGMENT_V0) > 0); - Assert.assertTrue(vAppend.compareTo(v1) > 0); + Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); final DataSegment segmentV10 = createSegment(FIRST_OF_JAN_23, v1); replaceTask.commitReplaceSegments(segmentV10); @@ -566,12 +561,17 @@ public void negativeTestLockAllocateReplaceDayAppendMonth_appendedSegmentOversha verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); - final DataSegment appendedSegment = asSegment(pendingSegment); - appendTask.commitAppendSegments(appendedSegment); + final DataSegment segmentV01 = asSegment(pendingSegment); + appendTask.commitAppendSegments(segmentV01); - // Fix this: Appended segment incorrectly overshadows replace segment - verifyIntervalHasUsedSegments(JAN_23, appendedSegment, segmentV10); - verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, appendedSegment); + final DataSegment segmentV11 = DataSegment.builder(segmentV01) + .interval(FIRST_OF_JAN_23) + .version(v1) + .shardSpec(new NumberedShardSpec(1, 1)) + .build(); + + verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV01, segmentV10, segmentV11); + verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10, segmentV11); } @Test From 5f5c5bfccda312b4d858104a6350e8c816ca70f4 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 22 Sep 2023 16:38:53 +0530 Subject: [PATCH 37/43] Fix IndexerSQLCoordinator tests --- .../indexing/common/actions/TaskLocks.java | 22 ++ .../common/task/AbstractBatchIndexTask.java | 57 +++- .../indexing/common/task/TaskLockHelper.java | 32 +- .../SinglePhaseParallelIndexTaskRunner.java | 19 +- .../overlord/LockRequestForNewSegment.java | 1 - .../druid/indexing/overlord/TaskLockbox.java | 1 - .../overlord/TimeChunkLockRequest.java | 8 +- ...TestIndexerMetadataStorageCoordinator.java | 1 - .../IndexerSQLMetadataStorageCoordinator.java | 132 ++++---- ...exerSQLMetadataStorageCoordinatorTest.java | 297 ++++++------------ 10 files changed, 237 insertions(+), 333 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java index 045b60495f02..b480990d6158 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java @@ -25,7 +25,9 @@ import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TimeChunkLock; +import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.overlord.ReplaceTaskLock; import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.java.util.common.DateTimes; @@ -125,6 +127,26 @@ public static boolean isLockCoversSegments( ); } + /** + * Determines the type of time chunk lock to use for appending segments. + *

+ * This method should be de-duplicated with {@link AbstractBatchIndexTask#determineLockType} + * by passing the ParallelIndexSupervisorTask instance into the + * SinglePhaseParallelIndexTaskRunner. + */ + public static TaskLockType determineLockTypeForAppend( + Map context + ) + { + final Object lockType = context.get(Tasks.TASK_LOCK_TYPE); + if (lockType == null) { + final boolean useSharedLock = (boolean) context.getOrDefault(Tasks.USE_SHARED_LOCK, false); + return useSharedLock ? TaskLockType.SHARED : TaskLockType.EXCLUSIVE; + } else { + return TaskLockType.valueOf(lockType.toString()); + } + } + /** * Finds locks of type {@link TaskLockType#REPLACE} for each of the given segments * that have an interval completely covering the interval of the respective segments. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index 148d09f302ce..c88d2274ca05 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -299,10 +299,10 @@ public boolean determineLockGranularityAndTryLock(TaskActionClient client, List< // Respect task context value most. if (forceTimeChunkLock || ingestionMode == IngestionMode.REPLACE) { log.info( - "forceTimeChunkLock[%s] is set to true or mode[%s] is replace. Use timeChunk lock", + "Using time chunk lock since forceTimeChunkLock is [%s] and mode is [%s].", forceTimeChunkLock, ingestionMode ); - taskLockHelper = new TaskLockHelper(false, getContext(), ingestionMode); + taskLockHelper = createLockHelper(LockGranularity.TIME_CHUNK); if (!intervals.isEmpty()) { return tryTimeChunkLock(client, intervals); } else { @@ -311,11 +311,7 @@ public boolean determineLockGranularityAndTryLock(TaskActionClient client, List< } else { if (!intervals.isEmpty()) { final LockGranularityDetermineResult result = determineSegmentGranularity(client, intervals); - taskLockHelper = new TaskLockHelper( - result.lockGranularity == LockGranularity.SEGMENT, - getContext(), - ingestionMode - ); + taskLockHelper = createLockHelper(result.lockGranularity); return tryLockWithDetermineResult(client, result); } else { // This branch is the only one that will not initialize taskLockHelper. @@ -343,11 +339,10 @@ boolean determineLockGranularityAndTryLockWithSegments( Tasks.FORCE_TIME_CHUNK_LOCK_KEY, Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK ); - final IngestionMode ingestionMode = getIngestionMode(); if (forceTimeChunkLock) { log.info("[%s] is set to true in task context. Use timeChunk lock", Tasks.FORCE_TIME_CHUNK_LOCK_KEY); - taskLockHelper = new TaskLockHelper(false, getContext(), ingestionMode); + taskLockHelper = createLockHelper(LockGranularity.TIME_CHUNK); segmentCheckFunction.accept(LockGranularity.TIME_CHUNK, segments); return tryTimeChunkLock( client, @@ -355,11 +350,7 @@ boolean determineLockGranularityAndTryLockWithSegments( ); } else { final LockGranularityDetermineResult result = determineSegmentGranularity(segments); - taskLockHelper = new TaskLockHelper( - result.lockGranularity == LockGranularity.SEGMENT, - getContext(), - ingestionMode - ); + taskLockHelper = createLockHelper(result.lockGranularity); segmentCheckFunction.accept(result.lockGranularity, segments); return tryLockWithDetermineResult(client, result); } @@ -435,7 +426,7 @@ protected boolean tryTimeChunkLock(TaskActionClient client, List inter } prev = cur; - final TaskLockType taskLockType = TaskLockHelper.lockTypeFrom(false, getContext(), getIngestionMode()); + final TaskLockType taskLockType = determineLockType(LockGranularity.TIME_CHUNK); final TaskLock lock = client.submit(new TimeChunkLockTryAcquireAction(taskLockType, cur)); if (lock == null) { return false; @@ -449,6 +440,42 @@ protected boolean tryTimeChunkLock(TaskActionClient client, List inter return true; } + private TaskLockHelper createLockHelper(LockGranularity lockGranularity) + { + return new TaskLockHelper( + lockGranularity == LockGranularity.SEGMENT, + determineLockType(lockGranularity) + ); + } + + /** + * Determines the type of lock to use with the given lock granularity. + */ + private TaskLockType determineLockType(LockGranularity lockGranularity) + { + if (lockGranularity == LockGranularity.SEGMENT) { + return TaskLockType.EXCLUSIVE; + } + + final String contextLockType = getContextValue(Tasks.TASK_LOCK_TYPE); + final TaskLockType lockType; + if (contextLockType == null) { + lockType = getContextValue(Tasks.USE_SHARED_LOCK, false) + ? TaskLockType.SHARED : TaskLockType.EXCLUSIVE; + } else { + lockType = TaskLockType.valueOf(contextLockType); + } + + final IngestionMode ingestionMode = getIngestionMode(); + if ((lockType == TaskLockType.SHARED || lockType == TaskLockType.APPEND) + && ingestionMode != IngestionMode.APPEND) { + // Lock types SHARED and APPEND are allowed only in APPEND ingestion mode + return Tasks.DEFAULT_TASK_LOCK_TYPE; + } else { + return lockType; + } + } + private LockGranularityDetermineResult determineSegmentGranularity(List segments) { if (segments.isEmpty()) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskLockHelper.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskLockHelper.java index 84f6f3e1f45e..cfe23a07bdc4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskLockHelper.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/TaskLockHelper.java @@ -20,7 +20,6 @@ package org.apache.druid.indexing.common.task; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableSet; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.actions.SegmentLockTryAcquireAction; @@ -92,10 +91,10 @@ public short getMinorVersionForNewSegments() } } - public TaskLockHelper(boolean useSegmentLock, Map context, AbstractTask.IngestionMode mode) + public TaskLockHelper(boolean useSegmentLock, TaskLockType taskLockType) { this.useSegmentLock = useSegmentLock; - this.taskLockType = lockTypeFrom(useSegmentLock, context, mode); + this.taskLockType = taskLockType; } public boolean isUseSegmentLock() @@ -311,31 +310,4 @@ public static void verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(List< ); } } - - public static TaskLockType lockTypeFrom( - boolean useSegmentLock, - Map context, - AbstractTask.IngestionMode mode - ) - { - if (useSegmentLock) { - return Tasks.DEFAULT_TASK_LOCK_TYPE; - } - - final TaskLockType lockTypeVal; - final Object lockTypeName = context.get(Tasks.TASK_LOCK_TYPE); - if (lockTypeName == null) { - final Object useSharedLock = context.getOrDefault(Tasks.USE_SHARED_LOCK, false); - lockTypeVal = (boolean) useSharedLock ? TaskLockType.SHARED : TaskLockType.EXCLUSIVE; - } else { - lockTypeVal = TaskLockType.valueOf(lockTypeName.toString()); - } - - final Set appendModeCompatible = ImmutableSet.of(TaskLockType.SHARED, TaskLockType.APPEND); - if (!mode.equals(AbstractTask.IngestionMode.APPEND) && appendModeCompatible.contains(lockTypeVal)) { - return Tasks.DEFAULT_TASK_LOCK_TYPE; - } else { - return lockTypeVal; - } - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java index 77b96ca6b216..76311c0dbb6d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java @@ -26,12 +26,12 @@ import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.indexing.common.Counters; +import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.TaskLocks; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; -import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.common.task.TaskLockHelper; import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskCompleteEvent; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.NonnullPair; @@ -206,10 +206,7 @@ SubTaskSpec newTaskSpec(InputSplit split) @Deprecated public SegmentIdWithShardSpec allocateNewSegment(String dataSource, DateTime timestamp) throws IOException { - NonnullPair intervalAndVersion = findIntervalAndVersion( - timestamp, - TaskLockHelper.lockTypeFrom(false, getContext(), AbstractTask.IngestionMode.APPEND) - ); + NonnullPair intervalAndVersion = findIntervalAndVersion(timestamp); final int partitionNum = Counters.getAndIncrementInt(partitionNumCountersPerInterval, intervalAndVersion.lhs); return new SegmentIdWithShardSpec( @@ -222,7 +219,7 @@ public SegmentIdWithShardSpec allocateNewSegment(String dataSource, DateTime tim /** * Allocate a new segment for the given timestamp locally. This method is called when dynamic partitioning is used - * and {@link org.apache.druid.indexing.common.LockGranularity} is {@code TIME_CHUNK}. + * and {@link LockGranularity} is {@code TIME_CHUNK}. * * The allocation algorithm is similar to the Overlord-based segment allocation. It keeps the segment allocation * history per sequenceName. If the prevSegmentId is found in the segment allocation history, this method @@ -239,10 +236,7 @@ public SegmentIdWithShardSpec allocateNewSegment( @Nullable String prevSegmentId ) throws IOException { - NonnullPair intervalAndVersion = findIntervalAndVersion( - timestamp, - TaskLockHelper.lockTypeFrom(false, getContext(), AbstractTask.IngestionMode.APPEND) - ); + NonnullPair intervalAndVersion = findIntervalAndVersion(timestamp); MutableObject segmentIdHolder = new MutableObject<>(); sequenceToSegmentIds.compute(sequenceName, (k, v) -> { @@ -290,8 +284,9 @@ public SegmentIdWithShardSpec allocateNewSegment( return segmentIdHolder.getValue(); } - NonnullPair findIntervalAndVersion(DateTime timestamp, TaskLockType taskLockType) throws IOException + NonnullPair findIntervalAndVersion(DateTime timestamp) throws IOException { + TaskLockType taskLockType = TaskLocks.determineLockTypeForAppend(getContext()); return AbstractBatchIndexTask.findIntervalAndVersion(getToolbox(), ingestionSchema, timestamp, taskLockType); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java index 7cf8966b0270..60a85413a4a9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java @@ -25,7 +25,6 @@ import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.actions.TaskLocks; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.timeline.partition.PartialShardSpec; import org.joda.time.Interval; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 6abe2eebc68c..8af9ff6f1caa 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -1311,7 +1311,6 @@ Map>>> ge return running; } - /** * Check if the lock for a given request can coexist with a given set of conflicting posses without any revocation. * @param conflictPosses conflict lock posses diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java index d92041c7ab04..9d2d604e94c9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java @@ -77,11 +77,7 @@ public TimeChunkLockRequest( this.groupId = groupId; this.dataSource = dataSource; this.interval = interval; - if (preferredVersion == null) { - this.preferredVersion = TaskLocks.defaultLockVersion(lockType); - } else { - this.preferredVersion = preferredVersion; - } + this.preferredVersion = preferredVersion; this.priority = priority; this.revoked = revoked; } @@ -119,7 +115,7 @@ public Interval getInterval() @Override public String getVersion() { - return preferredVersion; + return preferredVersion == null ? TaskLocks.defaultLockVersion(lockType) : preferredVersion; } @Override diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index 6d788e6c23d0..a97e788d7f49 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -163,7 +163,6 @@ public SegmentPublishResult commitReplaceSegments( Set locksHeldByReplaceTask ) { - // Don't actually compare metadata, just do it! return SegmentPublishResult.ok(commitSegments(replaceSegments)); } diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index cc9bec498e7e..06692607036f 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -63,7 +63,6 @@ import org.apache.druid.timeline.partition.PartitionIds; import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; -import org.apache.druid.utils.CollectionUtils; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.chrono.ISOChronology; @@ -443,7 +442,7 @@ public SegmentPublishResult commitAppendSegments( verifySegmentsToCommit(appendSegments); final String dataSource = appendSegments.iterator().next().getDataSource(); - final Map> segmentToNewIds = connector.retryTransaction( + final Set upgradedSegments = connector.retryTransaction( (handle, transactionStatus) -> getSegmentsToUpgradeOnAppend(handle, dataSource, appendSegments), 0, @@ -452,17 +451,7 @@ public SegmentPublishResult commitAppendSegments( // Create entries for all required versions of the append segments final Set allSegmentsToInsert = new HashSet<>(appendSegments); - for (Map.Entry> entry : segmentToNewIds.entrySet()) { - final DataSegment segment = entry.getKey(); - for (SegmentIdWithShardSpec newId : entry.getValue()) { - DataSegment newSegment = DataSegment.builder(segment) - .interval(newId.getInterval()) - .version(newId.getVersion()) - .shardSpec(newId.getShardSpec()) - .build(); - allSegmentsToInsert.add(newSegment); - } - } + allSegmentsToInsert.addAll(upgradedSegments); try { return connector.retryTransaction( @@ -1066,15 +1055,14 @@ private void insertPendingSegmentIntoMetastore( * there would be some used segments in the DB with versions higher than these * append segments. */ - @VisibleForTesting - Map> getSegmentsToUpgradeOnAppend( + private Set getSegmentsToUpgradeOnAppend( Handle handle, String dataSource, Set segmentsToAppend ) throws IOException { if (segmentsToAppend.isEmpty()) { - return Collections.emptyMap(); + return Collections.emptySet(); } final Set appendIntervals = new HashSet<>(); @@ -1101,7 +1089,7 @@ Map> getSegmentsToUpgradeOnAppend( .add(segment); } - final Map> appendSegmentToNewIds = new HashMap<>(); + final Set upgradedSegments = new HashSet<>(); for (String upgradeVersion : committedVersionToIntervals.keySet()) { Map> segmentsToUpgrade = getSegmentsWithVersionLowerThan( upgradeVersion, @@ -1109,18 +1097,19 @@ Map> getSegmentsToUpgradeOnAppend( appendVersionToSegments ); for (Map.Entry> entry : segmentsToUpgrade.entrySet()) { - computeNewAppendIdsForVersion( + Set segmentsUpgradedToVersion = upgradeSegmentsToVersion( handle, upgradeVersion, entry.getKey(), entry.getValue(), - committedIntervalToSegments, - appendSegmentToNewIds + committedIntervalToSegments ); + log.info("Upgraded [%d] segments to version[%s].", segmentsUpgradedToVersion.size(), upgradeVersion); + upgradedSegments.addAll(segmentsUpgradedToVersion); } } - return appendSegmentToNewIds; + return upgradedSegments; } /** @@ -1164,13 +1153,12 @@ private Map> getSegmentsWithVersionLowerThan( * Computes new Segment IDs for the {@code segmentsToUpgrade} being upgraded * to the given {@code upgradeVersion}. */ - private void computeNewAppendIdsForVersion( + private Set upgradeSegmentsToVersion( Handle handle, String upgradeVersion, Interval interval, Set segmentsToUpgrade, - Map> committedSegmentsByInterval, - Map> appendSegmentToNewIds + Map> committedSegmentsByInterval ) throws IOException { final Set committedSegments @@ -1194,6 +1182,7 @@ private void computeNewAppendIdsForVersion( // Determine new IDs for each append segment by taking into account both // committed and pending segments for this version + final Set upgradedSegments = new HashSet<>(); for (DataSegment segment : segmentsToUpgrade) { SegmentCreateRequest request = new SegmentCreateRequest( segment.getId() + "__" + upgradeVersion, @@ -1213,9 +1202,16 @@ private void computeNewAppendIdsForVersion( // Add to set of pending segments so that shard specs are computed taking the new id into account pendingSegments.add(newId); - appendSegmentToNewIds.computeIfAbsent(segment, s -> new HashSet<>()) - .add(newId); + upgradedSegments.add( + DataSegment.builder(segment) + .interval(newId.getInterval()) + .version(newId.getVersion()) + .shardSpec(newId.getShardSpec()) + .build() + ); } + + return upgradedSegments; } private Map createNewSegments( @@ -1676,6 +1672,12 @@ private Set getSegmentsToUpgradeOnReplace( final Set locksHeldByReplaceTask ) { + // If a REPLACE task has locked an interval, it would commit some segments + // (or at least tombstones) in that interval (except in LEGACY_REPLACE ingestion mode) + if (replaceSegments.isEmpty() || locksHeldByReplaceTask.isEmpty()) { + return Collections.emptySet(); + } + // For each replace interval, find the number of core partitions and total partitions final Map intervalToNumCorePartitions = new HashMap<>(); final Map intervalToCurrentPartitionNum = new HashMap<>(); @@ -1689,29 +1691,45 @@ private Set getSegmentsToUpgradeOnReplace( ); } - final Map carryForwardSegmentToLockVersion - = getAppendSegmentsCommittedDuringTask(handle, locksHeldByReplaceTask); + // Find the segments that need to be upgraded + final String taskId = locksHeldByReplaceTask.stream() + .map(ReplaceTaskLock::getSupervisorTaskId) + .findFirst().orElse(null); + final Map upgradeSegmentToLockVersion + = getAppendSegmentsCommittedDuringTask(handle, taskId); + final List segmentsToUpgrade + = retrieveSegmentsById(handle, upgradeSegmentToLockVersion.keySet()); + + if (segmentsToUpgrade.isEmpty()) { + return Collections.emptySet(); + } - final List carryForwardSegments - = retrieveSegmentsById(handle, carryForwardSegmentToLockVersion.keySet()); + final Set replaceIntervals = intervalToNumCorePartitions.keySet(); - final Set segmentsToInsert = new HashSet<>(); - for (DataSegment oldSegment : carryForwardSegments) { - Interval newInterval = oldSegment.getInterval(); - for (DataSegment segment : replaceSegments) { - final Interval segmentInterval = segment.getInterval(); - if (segmentInterval.contains(newInterval)) { - newInterval = segmentInterval; + final Set upgradedSegments = new HashSet<>(); + for (DataSegment oldSegment : segmentsToUpgrade) { + // Determine interval of the upgraded segment + final Interval oldInterval = oldSegment.getInterval(); + Interval newInterval = null; + for (Interval replaceInterval : replaceIntervals) { + if (replaceInterval.contains(oldInterval)) { + newInterval = replaceInterval; break; - } else if (segmentInterval.overlaps(newInterval)) { + } else if (replaceInterval.overlaps(oldInterval)) { throw new ISE( "Incompatible segment intervals for commit: [%s] and [%s].", - newInterval, segmentInterval + oldInterval, replaceInterval ); } } - // Compute shard spec for the new version of the segment + if (newInterval == null) { + // This can happen only if no replace interval contains this segment + // but a (revoked) REPLACE lock covers this segment + newInterval = oldInterval; + } + + // Compute shard spec of the upgraded segment final int partitionNum = intervalToCurrentPartitionNum.compute( newInterval, (i, value) -> value == null ? 0 : value + 1 @@ -1719,16 +1737,18 @@ private Set getSegmentsToUpgradeOnReplace( final int numCorePartitions = intervalToNumCorePartitions.get(newInterval); ShardSpec shardSpec = new NumberedShardSpec(partitionNum, numCorePartitions); - String lockVersion = carryForwardSegmentToLockVersion.get(oldSegment.getId().toString()); - DataSegment newSegment = DataSegment.builder(oldSegment) - .interval(newInterval) - .version(lockVersion) - .shardSpec(shardSpec) - .build(); - segmentsToInsert.add(newSegment); + // Create upgraded segment with the correct interval, version and shard spec + String lockVersion = upgradeSegmentToLockVersion.get(oldSegment.getId().toString()); + upgradedSegments.add( + DataSegment.builder(oldSegment) + .interval(newInterval) + .version(lockVersion) + .shardSpec(shardSpec) + .build() + ); } - return segmentsToInsert; + return upgradedSegments; } /** @@ -1905,30 +1925,24 @@ private String buildSqlToInsertSegments() /** * Finds the append segments that were covered by the given task REPLACE locks. - * These append segments must now be carried forward to the same version as - * the segments being committed by this replace task. + * These append segments must now be upgraded to the same version as the segments + * being committed by this replace task. * * @return Map from append Segment ID to REPLACE lock version */ - @VisibleForTesting - Map getAppendSegmentsCommittedDuringTask( + private Map getAppendSegmentsCommittedDuringTask( Handle handle, - Set replaceLocks + String taskId ) { - if (CollectionUtils.isNullOrEmpty(replaceLocks)) { - return Collections.emptyMap(); - } - final String sql = StringUtils.format( "SELECT segment_id, lock_version FROM %1$s WHERE task_id = :task_id", dbTables.getSegmentVersionsTable() ); - final String groupId = replaceLocks.iterator().next().getSupervisorTaskId(); ResultIterator> resultIterator = handle .createQuery(sql) - .bind("task_id", groupId) + .bind("task_id", taskId) .map( (index, r, ctx) -> Pair.of(r.getString("segment_id"), r.getString("lock_version")) ) diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index bf59b2228841..dca9c7df1d4e 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -34,12 +34,12 @@ 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.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.DimensionRangeShardSpec; import org.apache.druid.timeline.partition.HashBasedNumberedPartialShardSpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; @@ -51,8 +51,8 @@ import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartialShardSpec; import org.apache.druid.timeline.partition.PartitionIds; +import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; -import org.apache.druid.utils.CollectionUtils; import org.assertj.core.api.Assertions; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -63,6 +63,7 @@ import org.junit.Test; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.PreparedBatch; +import org.skife.jdbi.v2.ResultIterator; import org.skife.jdbi.v2.util.StringMapper; import java.io.IOException; @@ -464,11 +465,30 @@ private Boolean insertUsedSegments(Set dataSegments) ); } - private Map getAppendedSegmentIds(Set replaceLocks) + private Map getSegmentsCommittedDuringReplaceTask(String taskId) { - return derbyConnector.retryWithHandle( - handle -> coordinator.getAppendSegmentsCommittedDuringTask(handle, replaceLocks) - ); + final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentVersionsTable(); + return derbyConnector.retryWithHandle(handle -> { + final String sql = StringUtils.format( + "SELECT segment_id, lock_version FROM %1$s WHERE task_id = :task_id", + table + ); + + ResultIterator> resultIterator = handle + .createQuery(sql) + .bind("task_id", taskId) + .map( + (index, r, ctx) -> Pair.of(r.getString("segment_id"), r.getString("lock_version")) + ) + .iterator(); + + final Map segmentIdToLockVersion = new HashMap<>(); + while (resultIterator.hasNext()) { + Pair result = resultIterator.next(); + segmentIdToLockVersion.put(result.lhs, result.rhs); + } + return segmentIdToLockVersion; + }); } private Boolean insertIntoSegmentVersionsTable(Map segmentToTaskLockMap) @@ -505,234 +525,79 @@ private Boolean insertIntoSegmentVersionsTable(Map } @Test - public void testAllocateNewSegmentIds() + public void testCommitAppendSegments() { - final String v0 = "1970-01-01"; - final String v1 = "2023-01-03"; - final String v2 = "2023-02-01"; + final String v1 = "2023-01-01"; + final String v2 = "2023-01-02"; + final String v3 = "2023-01-03"; + final String lockVersion = "2024-01-01"; - final Set day1 = new HashSet<>(); - for (int i = 0; i < 10; i++) { - final DataSegment segment = new DataSegment( - "foo", - Intervals.of("2023-01-01/2023-01-02"), - v0, - ImmutableMap.of("path", "a-" + i), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new LinearShardSpec(i), - 9, - 100 - ); - day1.add(segment); - } - final Set day2 = new HashSet<>(); - for (int i = 0; i < 10; i++) { - final DataSegment segment = new DataSegment( - "foo", - Intervals.of("2023-01-02/2023-01-03"), - v0, - ImmutableMap.of("path", "b-" + i), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new LinearShardSpec(i), - 9, - 100 - ); - day2.add(segment); - } - final Set day3 = new HashSet<>(); - for (int i = 0; i < 10; i++) { - final DataSegment segment = new DataSegment( - "foo", - Intervals.of("2023-01-03/2023-01-04"), - v0, - ImmutableMap.of("path", "c-" + i), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new LinearShardSpec(i), - 9, - 100 - ); - day3.add(segment); - } - - final Set month2 = new HashSet<>(); - for (int i = 0; i < 10; i++) { - final DataSegment segment = new DataSegment( - "foo", - Intervals.of("2023-02-01/2023-03-01"), - v0, - ImmutableMap.of("path", "x-" + i), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new LinearShardSpec(i), - 9, - 100 - ); - month2.add(segment); - } - - final Set higherVersionUsedSegments = new HashSet<>(); - for (int i = 0; i < 10; i++) { - final DataSegment segment = new DataSegment( - "foo", - Intervals.of("2023-01-01/2023-01-02"), - v1, - ImmutableMap.of("path", "d-" + i), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new NumberedShardSpec(i, 5), - 9, - 100 - ); - higherVersionUsedSegments.add(segment); - } - for (int i = 0; i < 10; i++) { - final DataSegment segment = new DataSegment( - "foo", - Intervals.of("2023-01-02/2023-01-03"), - v1, - ImmutableMap.of("path", "e-" + i), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new NumberedShardSpec(i, 0), - 9, - 100 - ); - higherVersionUsedSegments.add(segment); - } - for (int i = 0; i < 10; i++) { - final DataSegment segment = new DataSegment( - "foo", - Intervals.of("2023-01-01/2023-02-01"), - v2, - ImmutableMap.of("path", "f-" + i), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new NumberedShardSpec(i, 10), - 9, - 100 - ); - higherVersionUsedSegments.add(segment); - } - insertUsedSegments(higherVersionUsedSegments); - - final Set segmentsToBeProcessed = new HashSet<>(); - final Set month1 = new HashSet<>(); - month1.addAll(day1); - month1.addAll(day2); - month1.addAll(day3); - segmentsToBeProcessed.addAll(month1); - segmentsToBeProcessed.addAll(month2); - final Map> segmentToNewIds = derbyConnector.retryWithHandle( - handle -> coordinator.getSegmentsToUpgradeOnAppend(handle, "foo", segmentsToBeProcessed) + final String replaceTaskId = "replaceTask1"; + final ReplaceTaskLock replaceLock = new ReplaceTaskLock( + replaceTaskId, + Intervals.of("2023-01-01/2023-01-03"), + lockVersion ); - for (DataSegment segment : day1) { - final Set newIds = segmentToNewIds.get(segment); - - Assert.assertEquals(2, newIds.size()); - Assert.assertEquals( - ImmutableSet.of(v1, v2), - newIds.stream().map(SegmentIdWithShardSpec::getVersion).collect(Collectors.toSet()) - ); - } - for (DataSegment segment : day2) { - final Set newIds = segmentToNewIds.get(segment); - - Assert.assertEquals(2, newIds.size()); - Assert.assertEquals( - ImmutableSet.of(v1, v2), - newIds.stream().map(SegmentIdWithShardSpec::getVersion).collect(Collectors.toSet()) - ); - } - for (DataSegment segment : day3) { - final Set newIds = segmentToNewIds.get(segment); - Assert.assertEquals(2, newIds.size()); - Assert.assertEquals( - ImmutableSet.of(v1, v2), - newIds.stream().map(SegmentIdWithShardSpec::getVersion).collect(Collectors.toSet()) - ); - } - for (DataSegment segment : month2) { - Assert.assertTrue(CollectionUtils.isNullOrEmpty(segmentToNewIds.get(segment))); - } - } - - @Test - public void testCommitAppendSegments() - { - final Set allSegments = new HashSet<>(); - final Set segmentIdsToBeCarriedForward = new HashSet<>(); - final ReplaceTaskLock lock = new ReplaceTaskLock("g1", Intervals.of("2023-01-01/2023-01-03"), "2024-01-01"); - final Map segmentLockMap = new HashMap<>(); - + final Set appendSegments = new HashSet<>(); + final Set expectedSegmentsToUpgrade = new HashSet<>(); for (int i = 0; i < 10; i++) { - final DataSegment segment = new DataSegment( - "foo", + final DataSegment segment = createSegment( Intervals.of("2023-01-01/2023-01-02"), - "2023-01-01", - ImmutableMap.of("path", "a-" + i), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new LinearShardSpec(i), - 9, - 100 + v1, + new LinearShardSpec(i) ); - allSegments.add(segment); - segmentIdsToBeCarriedForward.add(segment.getId().toString()); - segmentLockMap.put(segment, lock); + appendSegments.add(segment); + expectedSegmentsToUpgrade.add(segment); } for (int i = 0; i < 10; i++) { - final DataSegment segment = new DataSegment( - "foo", + final DataSegment segment = createSegment( Intervals.of("2023-01-02/2023-01-03"), - "2023-01-02", - ImmutableMap.of("path", "b-" + i), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new LinearShardSpec(i), - 9, - 100 + v2, + new LinearShardSpec(i) ); - allSegments.add(segment); - segmentIdsToBeCarriedForward.add(segment.getId().toString()); - segmentLockMap.put(segment, lock); + appendSegments.add(segment); + expectedSegmentsToUpgrade.add(segment); } for (int i = 0; i < 10; i++) { - final DataSegment segment = new DataSegment( - "foo", + final DataSegment segment = createSegment( Intervals.of("2023-01-03/2023-01-04"), - "2023-01-03", - ImmutableMap.of("path", "c-" + i), - ImmutableList.of("dim1"), - ImmutableList.of("m1"), - new LinearShardSpec(i), - 9, - 100 + v3, + new LinearShardSpec(i) ); - allSegments.add(segment); + appendSegments.add(segment); } - coordinator.commitAppendSegments(allSegments, segmentLockMap); + final Map segmentToReplaceLock = expectedSegmentsToUpgrade.stream().collect( + Collectors.toMap(s -> s, s -> replaceLock) + ); + + // Commit the segment and verify the results + SegmentPublishResult commitResult = coordinator.commitAppendSegments(appendSegments, segmentToReplaceLock); + Assert.assertTrue(commitResult.isSuccess()); + Assert.assertEquals(appendSegments, commitResult.getSegments()); + // Verify the segments present in the metadata store Assert.assertEquals( - allSegments.stream().map(DataSegment::getId).map(SegmentId::toString).collect(Collectors.toSet()), - ImmutableSet.copyOf(retrieveUsedSegmentIds()) + appendSegments, + ImmutableSet.copyOf(retrieveUsedSegments()) ); - final Set replaceLocks = Collections.singleton(lock); - final Map segmentLockMetadata = getAppendedSegmentIds(replaceLocks); - Assert.assertEquals(segmentIdsToBeCarriedForward, segmentLockMetadata.keySet()); + final Set expectedUpgradeSegmentIds = expectedSegmentsToUpgrade.stream() + .map(s -> s.getId().toString()) + .collect(Collectors.toSet()); + final Map observedSegmentToLock = getSegmentsCommittedDuringReplaceTask(replaceTaskId); + Assert.assertEquals(expectedUpgradeSegmentIds, observedSegmentToLock.keySet()); - final Set lockVersions = new HashSet<>(segmentLockMetadata.values()); - Assert.assertEquals(lock.getVersion(), Iterables.getOnlyElement(lockVersions)); + final Set observedLockVersions = new HashSet<>(observedSegmentToLock.values()); + Assert.assertEquals( + replaceLock.getVersion(), + Iterables.getOnlyElement(observedLockVersions) + ); } - @Test public void testCommitReplaceSegments() { @@ -2688,4 +2553,20 @@ public void testMarkSegmentsAsUnusedWithinIntervalTwoYears() throws IOException ) ); } + + private static class DS + { + static final String WIKI = "wiki"; + } + + private DataSegment createSegment(Interval interval, String version, ShardSpec shardSpec) + { + return DataSegment.builder() + .dataSource(DS.WIKI) + .interval(interval) + .version(version) + .shardSpec(shardSpec) + .size(100) + .build(); + } } From 444cfe4e7dfcdefe553952c1494b5c8a86261951 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sat, 23 Sep 2023 08:11:49 +0530 Subject: [PATCH 38/43] Add tests for new TaskLocks utility methods --- .../SegmentTransactionalAppendAction.java | 2 +- .../SegmentTransactionalReplaceAction.java | 2 +- .../indexing/common/actions/TaskLocks.java | 8 +- .../druid/indexing/overlord/TaskLockbox.java | 1 + .../common/actions/TaskLocksTest.java | 111 ++++++++++++++---- ...TestIndexerMetadataStorageCoordinator.java | 2 +- .../IndexerMetadataStorageCoordinator.java | 1 + .../IndexerSQLMetadataStorageCoordinator.java | 1 - .../ReplaceTaskLock.java | 2 +- ...exerSQLMetadataStorageCoordinatorTest.java | 1 - 10 files changed, 99 insertions(+), 32 deletions(-) rename server/src/main/java/org/apache/druid/{indexing/overlord => metadata}/ReplaceTaskLock.java (98%) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java index e6c1261d261a..171c4f6640f3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalAppendAction.java @@ -25,9 +25,9 @@ import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; -import org.apache.druid.indexing.overlord.ReplaceTaskLock; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.metadata.ReplaceTaskLock; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java index 6dd2fd9dbd6f..5a1228e1dd1d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalReplaceAction.java @@ -26,9 +26,9 @@ import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; -import org.apache.druid.indexing.overlord.ReplaceTaskLock; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.metadata.ReplaceTaskLock; import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java index b480990d6158..2f42ebb21c40 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java @@ -28,10 +28,10 @@ import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Tasks; -import org.apache.druid.indexing.overlord.ReplaceTaskLock; import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.metadata.ReplaceTaskLock; import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -135,12 +135,12 @@ public static boolean isLockCoversSegments( * SinglePhaseParallelIndexTaskRunner. */ public static TaskLockType determineLockTypeForAppend( - Map context + Map taskContext ) { - final Object lockType = context.get(Tasks.TASK_LOCK_TYPE); + final Object lockType = taskContext.get(Tasks.TASK_LOCK_TYPE); if (lockType == null) { - final boolean useSharedLock = (boolean) context.getOrDefault(Tasks.USE_SHARED_LOCK, false); + final boolean useSharedLock = (boolean) taskContext.getOrDefault(Tasks.USE_SHARED_LOCK, false); return useSharedLock ? TaskLockType.SHARED : TaskLockType.EXCLUSIVE; } else { return TaskLockType.valueOf(lockType.toString()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 8af9ff6f1caa..aaa563fa4e8c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -44,6 +44,7 @@ import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.metadata.ReplaceTaskLock; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java index cc3a02546523..43a403e59a90 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.common.actions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.druid.indexing.common.SegmentLock; import org.apache.druid.indexing.common.TaskLock; @@ -28,6 +29,7 @@ import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.LockResult; import org.apache.druid.indexing.overlord.SpecificSegmentLockRequest; @@ -36,6 +38,7 @@ import org.apache.druid.indexing.test.TestIndexerMetadataStorageCoordinator; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.metadata.ReplaceTaskLock; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; @@ -136,9 +139,13 @@ private Set createNumberedPartitionedSegments() ); } - private LockResult tryTimeChunkLock(Task task, Interval interval) + private TaskLock tryTimeChunkLock(Task task, Interval interval, TaskLockType lockType) { - return lockbox.tryLock(task, new TimeChunkLockRequest(TaskLockType.EXCLUSIVE, task, interval, null)); + final TaskLock taskLock = lockbox + .tryLock(task, new TimeChunkLockRequest(lockType, task, interval, null)) + .getTaskLock(); + Assert.assertNotNull(taskLock); + return taskLock; } private LockResult trySegmentLock(Task task, Interval interval, String version, int partitonId) @@ -162,11 +169,7 @@ public void testCheckLockCoversSegments() final Map locks = intervals.stream().collect( Collectors.toMap( Function.identity(), - interval -> { - final TaskLock lock = tryTimeChunkLock(task, interval).getTaskLock(); - Assert.assertNotNull(lock); - return lock; - } + interval -> tryTimeChunkLock(task, interval, TaskLockType.EXCLUSIVE) ) ); @@ -206,11 +209,7 @@ public void testCheckLargeLockCoversSegments() final Map locks = intervals.stream().collect( Collectors.toMap( Function.identity(), - interval -> { - final TaskLock lock = tryTimeChunkLock(task, interval).getTaskLock(); - Assert.assertNotNull(lock); - return lock; - } + interval -> tryTimeChunkLock(task, interval, TaskLockType.EXCLUSIVE) ) ); @@ -231,11 +230,7 @@ public void testCheckLockCoversSegmentsWithOverlappedIntervals() final Map locks = lockIntervals.stream().collect( Collectors.toMap( Function.identity(), - interval -> { - final TaskLock lock = tryTimeChunkLock(task, interval).getTaskLock(); - Assert.assertNotNull(lock); - return lock; - } + interval -> tryTimeChunkLock(task, interval, TaskLockType.EXCLUSIVE) ) ); @@ -256,11 +251,7 @@ public void testFindLocksForSegments() final Map locks = intervals.stream().collect( Collectors.toMap( Function.identity(), - interval -> { - final TaskLock lock = tryTimeChunkLock(task, interval).getTaskLock(); - Assert.assertNotNull(lock); - return lock; - } + interval -> tryTimeChunkLock(task, interval, TaskLockType.EXCLUSIVE) ) ); @@ -305,6 +296,82 @@ public void testFindSegmentLocksForSegments() ); } + @Test + public void testFindReplaceLocksCoveringSegments() + { + final Set segments = createTimeChunkedSegments(); + + final Map lockResults = segments.stream().collect( + Collectors.toMap( + segment -> segment, + segment -> tryTimeChunkLock(task, segment.getInterval(), TaskLockType.REPLACE) + ) + ); + + final Map observedLocks + = TaskLocks.findReplaceLocksCoveringSegments(task.getDataSource(), lockbox, segments); + Assert.assertEquals(segments.size(), observedLocks.size()); + for (DataSegment segment : segments) { + TaskLock lockFromResult = lockResults.get(segment); + Assert.assertEquals( + new ReplaceTaskLock(task.getId(), lockFromResult.getInterval(), lockFromResult.getVersion()), + observedLocks.get(segment) + ); + } + } + + @Test + public void testLockTypeForAppendWithLockTypeInContext() + { + Assert.assertEquals( + TaskLockType.REPLACE, + TaskLocks.determineLockTypeForAppend( + ImmutableMap.of(Tasks.TASK_LOCK_TYPE, "REPLACE") + ) + ); + Assert.assertEquals( + TaskLockType.APPEND, + TaskLocks.determineLockTypeForAppend( + ImmutableMap.of(Tasks.TASK_LOCK_TYPE, "APPEND") + ) + ); + Assert.assertEquals( + TaskLockType.SHARED, + TaskLocks.determineLockTypeForAppend( + ImmutableMap.of(Tasks.TASK_LOCK_TYPE, "SHARED") + ) + ); + Assert.assertEquals( + TaskLockType.EXCLUSIVE, + TaskLocks.determineLockTypeForAppend( + ImmutableMap.of(Tasks.TASK_LOCK_TYPE, "EXCLUSIVE", Tasks.USE_SHARED_LOCK, true) + ) + ); + } + + @Test + public void testLockTypeForAppendWithNoLockTypeInContext() + { + Assert.assertEquals( + TaskLockType.EXCLUSIVE, + TaskLocks.determineLockTypeForAppend( + ImmutableMap.of() + ) + ); + Assert.assertEquals( + TaskLockType.EXCLUSIVE, + TaskLocks.determineLockTypeForAppend( + ImmutableMap.of(Tasks.USE_SHARED_LOCK, false) + ) + ); + Assert.assertEquals( + TaskLockType.SHARED, + TaskLocks.determineLockTypeForAppend( + ImmutableMap.of(Tasks.USE_SHARED_LOCK, true) + ) + ); + } + private TimeChunkLock newTimeChunkLock(Interval interval, String version) { return new TimeChunkLock( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index a97e788d7f49..34d2e44552a7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -25,12 +25,12 @@ import com.google.common.collect.Sets; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -import org.apache.druid.indexing.overlord.ReplaceTaskLock; import org.apache.druid.indexing.overlord.SegmentCreateRequest; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.metadata.ReplaceTaskLock; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.PartialShardSpec; diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 65b5f1ce35c4..b6bfe7efc7d8 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.overlord; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.metadata.ReplaceTaskLock; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.PartialShardSpec; diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 06692607036f..550c5ad805b5 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -35,7 +35,6 @@ import org.apache.commons.lang.StringEscapeUtils; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -import org.apache.druid.indexing.overlord.ReplaceTaskLock; import org.apache.druid.indexing.overlord.SegmentCreateRequest; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/ReplaceTaskLock.java b/server/src/main/java/org/apache/druid/metadata/ReplaceTaskLock.java similarity index 98% rename from server/src/main/java/org/apache/druid/indexing/overlord/ReplaceTaskLock.java rename to server/src/main/java/org/apache/druid/metadata/ReplaceTaskLock.java index c34d98377565..bc2ed1955618 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/ReplaceTaskLock.java +++ b/server/src/main/java/org/apache/druid/metadata/ReplaceTaskLock.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.indexing.overlord; +package org.apache.druid.metadata; import org.joda.time.Interval; diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index dca9c7df1d4e..d3c701a7a1f2 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -27,7 +27,6 @@ import org.apache.druid.data.input.StringTuple; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.ObjectMetadata; -import org.apache.druid.indexing.overlord.ReplaceTaskLock; import org.apache.druid.indexing.overlord.SegmentCreateRequest; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; From 6309702ad670202fbc98da70b8a482027e4ca0aa Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sat, 23 Sep 2023 12:43:54 +0530 Subject: [PATCH 39/43] Add tests for ReplaceTaskLock and SqlSegMetaManagerProvider --- .../SqlSegmentsMetadataManagerProvider.java | 2 - ...exerSQLMetadataStorageCoordinatorTest.java | 23 +++--- .../druid/metadata/ReplaceTaskLockTest.java | 36 +++++++++ ...qlSegmentsMetadataManagerProviderTest.java | 78 +++++++++++++++++++ 4 files changed, 126 insertions(+), 13 deletions(-) create mode 100644 server/src/test/java/org/apache/druid/metadata/ReplaceTaskLockTest.java create mode 100644 server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java diff --git a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java index 53d6d7e56622..943fd5b203f8 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java @@ -17,7 +17,6 @@ * under the License. */ - package org.apache.druid.metadata; import com.fasterxml.jackson.databind.ObjectMapper; @@ -25,7 +24,6 @@ import com.google.inject.Inject; import org.apache.druid.java.util.common.lifecycle.Lifecycle; - public class SqlSegmentsMetadataManagerProvider implements SegmentsMetadataManagerProvider { private final ObjectMapper jsonMapper; diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index d3c701a7a1f2..5c5066495e30 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -569,12 +569,13 @@ public void testCommitAppendSegments() appendSegments.add(segment); } - final Map segmentToReplaceLock = expectedSegmentsToUpgrade.stream().collect( - Collectors.toMap(s -> s, s -> replaceLock) - ); + final Map segmentToReplaceLock + = expectedSegmentsToUpgrade.stream() + .collect(Collectors.toMap(s -> s, s -> replaceLock)); // Commit the segment and verify the results - SegmentPublishResult commitResult = coordinator.commitAppendSegments(appendSegments, segmentToReplaceLock); + SegmentPublishResult commitResult + = coordinator.commitAppendSegments(appendSegments, segmentToReplaceLock); Assert.assertTrue(commitResult.isSuccess()); Assert.assertEquals(appendSegments, commitResult.getSegments()); @@ -584,17 +585,17 @@ public void testCommitAppendSegments() ImmutableSet.copyOf(retrieveUsedSegments()) ); - final Set expectedUpgradeSegmentIds = expectedSegmentsToUpgrade.stream() - .map(s -> s.getId().toString()) - .collect(Collectors.toSet()); + // Verify entries in the segment task lock table + final Set expectedUpgradeSegmentIds + = expectedSegmentsToUpgrade.stream() + .map(s -> s.getId().toString()) + .collect(Collectors.toSet()); final Map observedSegmentToLock = getSegmentsCommittedDuringReplaceTask(replaceTaskId); Assert.assertEquals(expectedUpgradeSegmentIds, observedSegmentToLock.keySet()); final Set observedLockVersions = new HashSet<>(observedSegmentToLock.values()); - Assert.assertEquals( - replaceLock.getVersion(), - Iterables.getOnlyElement(observedLockVersions) - ); + Assert.assertEquals(1, observedLockVersions.size()); + Assert.assertEquals(replaceLock.getVersion(), Iterables.getOnlyElement(observedLockVersions)); } @Test diff --git a/server/src/test/java/org/apache/druid/metadata/ReplaceTaskLockTest.java b/server/src/test/java/org/apache/druid/metadata/ReplaceTaskLockTest.java new file mode 100644 index 000000000000..1d03a52c1454 --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/ReplaceTaskLockTest.java @@ -0,0 +1,36 @@ +/* + * 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 nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class ReplaceTaskLockTest +{ + + @Test + public void testEqualsAndHashCode() + { + EqualsVerifier.forClass(ReplaceTaskLock.class) + .usingGetClass() + .withNonnullFields("supervisorTaskId", "interval", "version") + .verify(); + } +} diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java new file mode 100644 index 000000000000..d1be4e93fcaa --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java @@ -0,0 +1,78 @@ +/* + * 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.common.base.Suppliers; +import org.apache.druid.java.util.common.lifecycle.Lifecycle; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; + +import java.util.Locale; + +public class SqlSegmentsMetadataManagerProviderTest +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule + = new TestDerbyConnector.DerbyConnectorRule(); + + private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + + @Test + public void testLifecycleStartCreatesSegmentTables() throws Exception + { + final TestDerbyConnector connector = derbyConnectorRule.getConnector(); + final SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(); + final Lifecycle lifecycle = new Lifecycle(); + + SqlSegmentsMetadataManagerProvider provider = new SqlSegmentsMetadataManagerProvider( + jsonMapper, + Suppliers.ofInstance(config), + derbyConnectorRule.metadataTablesConfigSupplier(), + connector, + lifecycle + ); + SegmentsMetadataManager manager = provider.get(); + Assert.assertTrue(manager instanceof SqlSegmentsMetadataManager); + + final MetadataStorageTablesConfig storageConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); + final String segmentsTable = storageConfig.getSegmentsTable().toUpperCase(Locale.ENGLISH); + final String segmentTaskLockTable = storageConfig.getSegmentVersionsTable().toUpperCase(Locale.ENGLISH); + + // Verify that the tables do not exist yet + Assert.assertFalse(tableExists(segmentsTable, connector)); + Assert.assertFalse(tableExists(segmentTaskLockTable, connector)); + + lifecycle.start(); + + // Verify that tables have now been created + Assert.assertTrue(tableExists(segmentsTable, connector)); + Assert.assertTrue(tableExists(segmentTaskLockTable, connector)); + + lifecycle.stop(); + } + + private boolean tableExists(String tableName, TestDerbyConnector connector) + { + return connector.retryWithHandle(handle -> connector.tableExists(handle, tableName)); + } +} \ No newline at end of file From 58433d0c47d6b3cfc80b1d5df643f174f4bb8bb5 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 24 Sep 2023 18:22:11 +0530 Subject: [PATCH 40/43] Add CreateTablesTest, rename new table to upgradeSegments --- .../metadata/MetadataStorageConnector.java | 2 +- .../metadata/MetadataStorageTablesConfig.java | 12 +-- .../MetadataStorageTablesConfigTest.java | 2 +- .../TestMetadataStorageConnector.java | 2 +- .../test/resources/test.runtime.properties | 2 +- .../IndexerMetadataStorageCoordinator.java | 40 +++++--- .../IndexerSQLMetadataStorageCoordinator.java | 14 +-- .../druid/metadata/SQLMetadataConnector.java | 6 +- .../SqlSegmentsMetadataManagerProvider.java | 2 +- ...exerSQLMetadataStorageCoordinatorTest.java | 14 +-- ...qlSegmentsMetadataManagerProviderTest.java | 4 +- .../org/apache/druid/cli/CreateTables.java | 2 +- .../apache/druid/cli/CreateTablesTest.java | 97 +++++++++++++++++++ 13 files changed, 157 insertions(+), 42 deletions(-) create mode 100644 services/src/test/java/org/apache/druid/cli/CreateTablesTest.java diff --git a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java index 932115e879e3..911b6d6bc611 100644 --- a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java @@ -77,7 +77,7 @@ default void exportTable( void createSegmentTable(); - void createSegmentVersionTable(); + void createUpgradeSegmentsTable(); void createRulesTable(); diff --git a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java index bc3203b93c4e..e9dc41ec1e11 100644 --- a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java +++ b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java @@ -57,8 +57,8 @@ public static MetadataStorageTablesConfig fromBase(String base) @JsonProperty("segments") private final String segmentsTable; - @JsonProperty("segmentVersions") - private final String segmentVersionsTable; + @JsonProperty("upgradeSegments") + private final String upgradeSegmentsTable; @JsonProperty("rules") private final String rulesTable; @@ -94,14 +94,14 @@ public MetadataStorageTablesConfig( @JsonProperty("taskLock") String taskLockTable, @JsonProperty("audit") String auditTable, @JsonProperty("supervisors") String supervisorTable, - @JsonProperty("segmentVersions") String segmentVersionsTable + @JsonProperty("upgradeSegments") String upgradeSegmentsTable ) { this.base = (base == null) ? DEFAULT_BASE : base; this.dataSourceTable = makeTableName(dataSourceTable, "dataSource"); this.pendingSegmentsTable = makeTableName(pendingSegmentsTable, "pendingSegments"); this.segmentsTable = makeTableName(segmentsTable, "segments"); - this.segmentVersionsTable = makeTableName(segmentVersionsTable, "segmentVersions"); + this.upgradeSegmentsTable = makeTableName(upgradeSegmentsTable, "upgradeSegments"); this.rulesTable = makeTableName(rulesTable, "rules"); this.configTable = makeTableName(configTable, "config"); @@ -147,9 +147,9 @@ public String getSegmentsTable() return segmentsTable; } - public String getSegmentVersionsTable() + public String getUpgradeSegmentsTable() { - return segmentVersionsTable; + return upgradeSegmentsTable; } public String getRulesTable() diff --git a/processing/src/test/java/org/apache/druid/guice/MetadataStorageTablesConfigTest.java b/processing/src/test/java/org/apache/druid/guice/MetadataStorageTablesConfigTest.java index c48570bb0c7a..242a2cf10334 100644 --- a/processing/src/test/java/org/apache/druid/guice/MetadataStorageTablesConfigTest.java +++ b/processing/src/test/java/org/apache/druid/guice/MetadataStorageTablesConfigTest.java @@ -80,6 +80,6 @@ public ObjectMapper jsonMapper() ); Assert.assertEquals(props.getProperty("druid.metadata.storage.tables.dataSource"), config.getDataSourceTable()); Assert.assertEquals(props.getProperty("druid.metadata.storage.tables.supervisors"), config.getSupervisorTable()); - Assert.assertEquals(props.getProperty("druid.metadata.storage.tables.segmentVersions"), config.getSegmentVersionsTable()); + Assert.assertEquals(props.getProperty("druid.metadata.storage.tables.upgradeSegments"), config.getUpgradeSegmentsTable()); } } diff --git a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java index 6edda9949957..3c98e6bcdddc 100644 --- a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java +++ b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java @@ -55,7 +55,7 @@ public void createSegmentTable() } @Override - public void createSegmentVersionTable() + public void createUpgradeSegmentsTable() { throw new UnsupportedOperationException(); } diff --git a/processing/src/test/resources/test.runtime.properties b/processing/src/test/resources/test.runtime.properties index 80ed65f863c5..f3af08d0f92e 100644 --- a/processing/src/test/resources/test.runtime.properties +++ b/processing/src/test/resources/test.runtime.properties @@ -27,6 +27,6 @@ druid.metadata.storage.tables.taskLock=fff_tasklock druid.metadata.storage.tables.audit=ggg_audit druid.metadata.storage.tables.dataSource=hhh_dataSource druid.metadata.storage.tables.supervisors=iii_supervisors -druid.metadata.storage.tables.segmentVersions=jjj_segmentVersions +druid.metadata.storage.tables.upgradeSegments=jjj_upgradeSegments druid.query.segmentMetadata.defaultAnalysisTypes=["cardinality", "size"] druid.query.segmentMetadata.defaultHistory=P2W diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index b6bfe7efc7d8..fb773c51eb68 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -279,13 +279,22 @@ SegmentPublishResult commitSegmentsAndMetadata( ) throws IOException; /** - * Commits segments created by an APPEND task. If any REPLACE segment that - * overlaps with these {@code appendSegments} was committed while this append - * task was in progress, the {@code appendSegments} are also added to the - * version of the replace segment. - * - * @param appendSegments - * @param appendSegmentToReplaceLock + * Commits segments created by an APPEND task. This method also handles the + * segment upgrade scenarios that may result from concurrent append and replace. + *

    + *
  • If a REPLACE task committed a segment that overlaps with any of the + * appendSegments while this APPEND task was in progress, the appendSegments + * are upgraded to the version of the replace segment too.
  • + *
  • If an appendSegment is covered by a currently active REPLACE lock, then + * an entry is created for it in the upgrade_segments table, so that when the + * REPLACE task finishes, it can upgrade the appendSegment to the version of + * the REPLACE lock.
  • + *
+ * + * @param appendSegments All segments created by an APPEND task that + * must be committed in a single transaction. + * @param appendSegmentToReplaceLock Map from append segment to the currently + * active REPLACE lock (if any) covering it */ SegmentPublishResult commitAppendSegments( Set appendSegments, @@ -293,10 +302,19 @@ SegmentPublishResult commitAppendSegments( ); /** - * Commits segments created by a REPLACE task. If any APPEND segment that - * overlaps with these {@code replaceSegments} was committed while this replace - * task was in progress, the append segments are also added to the version of - * these {@code replaceSegments}. + * Commits segments created by a REPLACE task. This method also handles the + * segment upgrade scenarios that may result from concurrent append and replace. + *
    + *
  • If an APPEND task committed a segment covered by a REPLACE lock of this + * task while it was in progress, the append segment is upgraded to the version + * of the corresponding lock. This is done with the help of entries created in + * upgrade_segments table in {@link #commitAppendSegments}
  • + *
  • + *
+ * + * @param replaceSegments All segments created by a REPLACE task that + * must be committed in a single transaction. + * @param locksHeldByReplaceTask All active non-revoked REPLACE locks held by the task */ SegmentPublishResult commitReplaceSegments( Set replaceSegments, diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 550c5ad805b5..9dfb8b7db68d 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -127,7 +127,7 @@ public void start() connector.createDataSourceTable(); connector.createPendingSegmentsTable(); connector.createSegmentTable(); - connector.createSegmentVersionTable(); + connector.createUpgradeSegmentsTable(); } @Override @@ -455,7 +455,7 @@ public SegmentPublishResult commitAppendSegments( try { return connector.retryTransaction( (handle, transactionStatus) -> { - insertSegmentLockVersions(handle, appendSegmentToReplaceLock); + insertIntoUpgradeSegmentsTable(handle, appendSegmentToReplaceLock); return SegmentPublishResult.ok(insertSegments(handle, allSegmentsToInsert)); }, 3, @@ -1834,10 +1834,10 @@ private Set insertSegments(Handle handle, Set segments } /** - * Inserts entries into the segment lock version table in batches of size + * Inserts entries into the upgrade_segments table in batches of size * {@link #MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE}. */ - private void insertSegmentLockVersions( + private void insertIntoUpgradeSegmentsTable( Handle handle, Map segmentToReplaceLock ) @@ -1850,7 +1850,7 @@ private void insertSegmentLockVersions( StringUtils.format( "INSERT INTO %1$s (task_id, segment_id, lock_version)" + " VALUES (:task_id, :segment_id, :lock_version)", - dbTables.getSegmentVersionsTable() + dbTables.getUpgradeSegmentsTable() ) ); @@ -1877,7 +1877,7 @@ private void insertSegmentLockVersions( } if (failedInserts.size() > 0) { throw new ISE( - "Failed to insert lock versions in DB: %s", + "Failed to insert upgrade segments in DB: %s", SegmentUtils.commaSeparatedIdentifiers(failedInserts) ); } @@ -1936,7 +1936,7 @@ private Map getAppendSegmentsCommittedDuringTask( { final String sql = StringUtils.format( "SELECT segment_id, lock_version FROM %1$s WHERE task_id = :task_id", - dbTables.getSegmentVersionsTable() + dbTables.getUpgradeSegmentsTable() ); ResultIterator> resultIterator = handle diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index 69a81b1d1913..3f1b6218fa6b 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -340,7 +340,7 @@ tableName, getPayloadType(), getQuoteString(), getCollation() ); } - private void createSegmentVersionTable(final String tableName) + private void createUpgradeSegments(final String tableName) { createTable( tableName, @@ -693,10 +693,10 @@ public void createSegmentTable() } @Override - public void createSegmentVersionTable() + public void createUpgradeSegmentsTable() { if (config.get().isCreateTables()) { - createSegmentVersionTable(tablesConfigSupplier.get().getSegmentVersionsTable()); + createUpgradeSegments(tablesConfigSupplier.get().getUpgradeSegmentsTable()); } } diff --git a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java index 943fd5b203f8..6a61e76d16b7 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java @@ -58,7 +58,7 @@ public SegmentsMetadataManager get() public void start() { connector.createSegmentTable(); - connector.createSegmentVersionTable(); + connector.createUpgradeSegmentsTable(); } @Override diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 5c5066495e30..4d1ccf381296 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -332,7 +332,7 @@ public void setUp() derbyConnector.createDataSourceTable(); derbyConnector.createTaskTables(); derbyConnector.createSegmentTable(); - derbyConnector.createSegmentVersionTable(); + derbyConnector.createUpgradeSegmentsTable(); derbyConnector.createPendingSegmentsTable(); metadataUpdateCounter.set(0); segmentTableDropUpdateCounter.set(0); @@ -466,7 +466,7 @@ private Boolean insertUsedSegments(Set dataSegments) private Map getSegmentsCommittedDuringReplaceTask(String taskId) { - final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentVersionsTable(); + final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getUpgradeSegmentsTable(); return derbyConnector.retryWithHandle(handle -> { final String sql = StringUtils.format( "SELECT segment_id, lock_version FROM %1$s WHERE task_id = :task_id", @@ -490,10 +490,10 @@ private Map getSegmentsCommittedDuringReplaceTask(String taskId) }); } - private Boolean insertIntoSegmentVersionsTable(Map segmentToTaskLockMap) + private void insertIntoUpgradeSegmentsTable(Map segmentToTaskLockMap) { - final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentVersionsTable(); - return derbyConnector.retryWithHandle( + final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getUpgradeSegmentsTable(); + derbyConnector.retryWithHandle( handle -> { PreparedBatch preparedBatch = handle.prepareBatch( StringUtils.format( @@ -516,7 +516,7 @@ private Boolean insertIntoSegmentVersionsTable(Map final int[] affectedRows = preparedBatch.execute(); final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); if (!succeeded) { - throw new ISE("Failed to publish segment to lock metadata mapping to DB"); + throw new ISE("Failed to insert upgrade segments in DB"); } return true; } @@ -620,7 +620,7 @@ public void testCommitReplaceSegments() appendedSegmentToReplaceLockMap.put(segment, replaceLock); } insertUsedSegments(segmentsAppendedWithReplaceLock); - insertIntoSegmentVersionsTable(appendedSegmentToReplaceLockMap); + insertIntoUpgradeSegmentsTable(appendedSegmentToReplaceLockMap); final Set replacingSegments = new HashSet<>(); for (int i = 1; i < 9; i++) { diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java index d1be4e93fcaa..cb7ef094094b 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java @@ -56,7 +56,7 @@ public void testLifecycleStartCreatesSegmentTables() throws Exception final MetadataStorageTablesConfig storageConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); final String segmentsTable = storageConfig.getSegmentsTable().toUpperCase(Locale.ENGLISH); - final String segmentTaskLockTable = storageConfig.getSegmentVersionsTable().toUpperCase(Locale.ENGLISH); + final String segmentTaskLockTable = storageConfig.getUpgradeSegmentsTable().toUpperCase(Locale.ENGLISH); // Verify that the tables do not exist yet Assert.assertFalse(tableExists(segmentsTable, connector)); @@ -75,4 +75,4 @@ private boolean tableExists(String tableName, TestDerbyConnector connector) { return connector.retryWithHandle(handle -> connector.tableExists(handle, tableName)); } -} \ No newline at end of file +} diff --git a/services/src/main/java/org/apache/druid/cli/CreateTables.java b/services/src/main/java/org/apache/druid/cli/CreateTables.java index 134eb5380b85..b6b37417ccaf 100644 --- a/services/src/main/java/org/apache/druid/cli/CreateTables.java +++ b/services/src/main/java/org/apache/druid/cli/CreateTables.java @@ -124,7 +124,7 @@ public void run() dbConnector.createDataSourceTable(); dbConnector.createPendingSegmentsTable(); dbConnector.createSegmentTable(); - dbConnector.createSegmentVersionTable(); + dbConnector.createUpgradeSegmentsTable(); dbConnector.createRulesTable(); dbConnector.createConfigTable(); dbConnector.createTaskTables(); diff --git a/services/src/test/java/org/apache/druid/cli/CreateTablesTest.java b/services/src/test/java/org/apache/druid/cli/CreateTablesTest.java new file mode 100644 index 000000000000..957daf50147f --- /dev/null +++ b/services/src/test/java/org/apache/druid/cli/CreateTablesTest.java @@ -0,0 +1,97 @@ +/* + * 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.cli; + +import com.google.inject.Injector; +import org.apache.druid.metadata.MetadataStorageConnector; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.TestDerbyConnector; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.Locale; + +public class CreateTablesTest +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule + = new TestDerbyConnector.DerbyConnectorRule(); + + private TestDerbyConnector connector; + + @Before + public void setup() + { + this.connector = derbyConnectorRule.getConnector(); + } + + @Test + public void testRunCreatesAllTables() + { + final MetadataStorageTablesConfig config = derbyConnectorRule.metadataTablesConfigSupplier().get(); + Assert.assertNotNull(config); + + // Verify that tables do not exist before starting + Assert.assertFalse(tableExists(config.getDataSourceTable())); + Assert.assertFalse(tableExists(config.getSegmentsTable())); + Assert.assertFalse(tableExists(config.getPendingSegmentsTable())); + Assert.assertFalse(tableExists(config.getUpgradeSegmentsTable())); + Assert.assertFalse(tableExists(config.getConfigTable())); + Assert.assertFalse(tableExists(config.getRulesTable())); + Assert.assertFalse(tableExists(config.getAuditTable())); + Assert.assertFalse(tableExists(config.getSupervisorTable())); + Assert.assertFalse(tableExists(config.getTaskLockTable())); + + // Run CreateTables + CreateTables createTables = new CreateTables() + { + @Override + public Injector makeInjector() + { + Injector injector = Mockito.mock(Injector.class); + Mockito.when(injector.getInstance(MetadataStorageConnector.class)).thenReturn(connector); + return injector; + } + }; + createTables.run(); + + // Verify that tables have now been created + Assert.assertTrue(tableExists(config.getDataSourceTable())); + Assert.assertTrue(tableExists(config.getSegmentsTable())); + Assert.assertTrue(tableExists(config.getPendingSegmentsTable())); + Assert.assertTrue(tableExists(config.getUpgradeSegmentsTable())); + Assert.assertTrue(tableExists(config.getConfigTable())); + Assert.assertTrue(tableExists(config.getRulesTable())); + Assert.assertTrue(tableExists(config.getAuditTable())); + Assert.assertTrue(tableExists(config.getSupervisorTable())); + Assert.assertTrue(tableExists(config.getTaskLockTable())); + } + + private boolean tableExists(String tableName) + { + return connector.retryWithHandle( + handle -> connector.tableExists(handle, tableName.toUpperCase(Locale.ENGLISH)) + ); + } + +} From a88da61a49ca06508de1396637eac00821304def Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 24 Sep 2023 20:56:57 +0530 Subject: [PATCH 41/43] Cleanup --- .../ConcurrentReplaceAndAppendTest.java | 14 -------------- .../IndexerMetadataStorageCoordinator.java | 18 ++++++++---------- .../druid/metadata/SQLMetadataConnector.java | 4 ++-- ...dexerSQLMetadataStorageCoordinatorTest.java | 2 +- ...SqlSegmentsMetadataManagerProviderTest.java | 12 +++++++----- services/pom.xml | 5 +++++ 6 files changed, 23 insertions(+), 32 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java index b78c7af2d321..293503b1c723 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java @@ -165,7 +165,6 @@ public void testLockReplaceAllocateAppend() replaceTask.commitReplaceSegments(segmentV10); verifyIntervalHasUsedSegments(FIRST_OF_JAN_23, segmentV10); - appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); Assert.assertEquals(segmentV10.getVersion(), pendingSegment.getVersion()); @@ -182,7 +181,6 @@ public void testLockAllocateAppendDayReplaceDay() { final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); - appendTask.acquireAppendLockOn(FIRST_OF_JAN_23).getVersion(); final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); @@ -209,7 +207,6 @@ public void testLockAllocateReplaceDayAppendDay() { final String v1 = replaceTask.acquireReplaceLockOn(FIRST_OF_JAN_23).getVersion(); - appendTask.acquireAppendLockOn(FIRST_OF_JAN_23).getVersion(); final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); @@ -234,7 +231,6 @@ public void testLockAllocateReplaceDayAppendDay() @Test public void testAllocateLockReplaceDayAppendDay() { - appendTask.acquireAppendLockOn(FIRST_OF_JAN_23).getVersion(); final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); @@ -261,7 +257,6 @@ public void testAllocateLockReplaceDayAppendDay() @Test public void testAllocateLockAppendDayReplaceDay() { - appendTask.acquireAppendLockOn(FIRST_OF_JAN_23).getVersion(); final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); @@ -289,7 +284,6 @@ public void testAllocateLockAppendDayReplaceDay() @Test public void testAllocateAppendDayLockReplaceDay() { - appendTask.acquireAppendLockOn(FIRST_OF_JAN_23).getVersion(); final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); Assert.assertEquals(SEGMENT_V0, pendingSegment.getVersion()); @@ -321,7 +315,6 @@ public void testLockReplaceMonthAllocateAppendDay() verifyIntervalHasVisibleSegments(FIRST_OF_JAN_23, segmentV10); // Verify that the allocated segment takes the version and interval of previous replace - appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); Assert.assertEquals(JAN_23, pendingSegment.getInterval()); @@ -339,7 +332,6 @@ public void testLockAllocateAppendDayReplaceMonth() { final String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); - appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); @@ -369,7 +361,6 @@ public void testLockAllocateReplaceMonthAppendDay() { final String v1 = replaceTask.acquireReplaceLockOn(JAN_23).getVersion(); - appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); @@ -397,7 +388,6 @@ public void testLockAllocateReplaceMonthAppendDay() @Test public void testAllocateLockReplaceMonthAppendDay() { - appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); @@ -427,7 +417,6 @@ public void testAllocateLockReplaceMonthAppendDay() @Test public void testAllocateLockAppendDayReplaceMonth() { - appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); @@ -457,7 +446,6 @@ public void testAllocateLockAppendDayReplaceMonth() @Test public void testAllocateAppendDayLockReplaceMonth() { - appendTask.acquireAppendLockOn(FIRST_OF_JAN_23); final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(FIRST_OF_JAN_23.getStart(), Granularities.DAY); Assert.assertEquals(FIRST_OF_JAN_23, pendingSegment.getInterval()); @@ -577,7 +565,6 @@ public void testLockAllocateReplaceDayAppendMonth() @Test public void testAllocateLockReplaceDayAppendMonth() { - appendTask.acquireAppendLockOn(JAN_23); final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); Assert.assertEquals(JAN_23, pendingSegment.getInterval()); @@ -608,7 +595,6 @@ public void testAllocateLockReplaceDayAppendMonth() @Test public void testAllocateAppendMonthLockReplaceDay() { - appendTask.acquireAppendLockOn(JAN_23); final SegmentIdWithShardSpec pendingSegment = appendTask.allocateSegmentForTimestamp(JAN_23.getStart(), Granularities.MONTH); Assert.assertEquals(JAN_23, pendingSegment.getInterval()); diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index fb773c51eb68..3cbabea78fae 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -279,16 +279,15 @@ SegmentPublishResult commitSegmentsAndMetadata( ) throws IOException; /** - * Commits segments created by an APPEND task. This method also handles the - * segment upgrade scenarios that may result from concurrent append and replace. + * Commits segments created by an APPEND task. This method also handles segment + * upgrade scenarios that may result from concurrent append and replace. *
    *
  • If a REPLACE task committed a segment that overlaps with any of the * appendSegments while this APPEND task was in progress, the appendSegments - * are upgraded to the version of the replace segment too.
  • + * are upgraded to the version of the replace segment. *
  • If an appendSegment is covered by a currently active REPLACE lock, then * an entry is created for it in the upgrade_segments table, so that when the - * REPLACE task finishes, it can upgrade the appendSegment to the version of - * the REPLACE lock.
  • + * REPLACE task finishes, it can upgrade the appendSegment as required. *
* * @param appendSegments All segments created by an APPEND task that @@ -305,11 +304,10 @@ SegmentPublishResult commitAppendSegments( * Commits segments created by a REPLACE task. This method also handles the * segment upgrade scenarios that may result from concurrent append and replace. *
    - *
  • If an APPEND task committed a segment covered by a REPLACE lock of this - * task while it was in progress, the append segment is upgraded to the version - * of the corresponding lock. This is done with the help of entries created in - * upgrade_segments table in {@link #commitAppendSegments}
  • - *
  • + *
  • If an APPEND task committed a segment to an interval locked by this task, + * the append segment is upgraded to the version of the corresponding lock. + * This is done with the help of entries created in the upgrade_segments table + * in {@link #commitAppendSegments}
  • *
* * @param replaceSegments All segments created by a REPLACE task that diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index 3f1b6218fa6b..6feaf9e07a38 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -340,7 +340,7 @@ tableName, getPayloadType(), getQuoteString(), getCollation() ); } - private void createUpgradeSegments(final String tableName) + private void createUpgradeSegmentsTable(final String tableName) { createTable( tableName, @@ -696,7 +696,7 @@ public void createSegmentTable() public void createUpgradeSegmentsTable() { if (config.get().isCreateTables()) { - createUpgradeSegments(tablesConfigSupplier.get().getUpgradeSegmentsTable()); + createUpgradeSegmentsTable(tablesConfigSupplier.get().getUpgradeSegmentsTable()); } } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 4d1ccf381296..b1b6f3aa16ea 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -641,7 +641,7 @@ public void testCommitReplaceSegments() coordinator.commitReplaceSegments(replacingSegments, ImmutableSet.of(replaceLock)); Assert.assertEquals( - 2 * segmentsAppendedWithReplaceLock.size() + replacingSegments.size(), + 2L * segmentsAppendedWithReplaceLock.size() + replacingSegments.size(), retrieveUsedSegmentIds().size() ); diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java index cb7ef094094b..04f49702783a 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProviderTest.java @@ -55,24 +55,26 @@ public void testLifecycleStartCreatesSegmentTables() throws Exception Assert.assertTrue(manager instanceof SqlSegmentsMetadataManager); final MetadataStorageTablesConfig storageConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); - final String segmentsTable = storageConfig.getSegmentsTable().toUpperCase(Locale.ENGLISH); - final String segmentTaskLockTable = storageConfig.getUpgradeSegmentsTable().toUpperCase(Locale.ENGLISH); + final String segmentsTable = storageConfig.getSegmentsTable(); + final String upgradeSegmentsTable = storageConfig.getUpgradeSegmentsTable(); // Verify that the tables do not exist yet Assert.assertFalse(tableExists(segmentsTable, connector)); - Assert.assertFalse(tableExists(segmentTaskLockTable, connector)); + Assert.assertFalse(tableExists(upgradeSegmentsTable, connector)); lifecycle.start(); // Verify that tables have now been created Assert.assertTrue(tableExists(segmentsTable, connector)); - Assert.assertTrue(tableExists(segmentTaskLockTable, connector)); + Assert.assertTrue(tableExists(upgradeSegmentsTable, connector)); lifecycle.stop(); } private boolean tableExists(String tableName, TestDerbyConnector connector) { - return connector.retryWithHandle(handle -> connector.tableExists(handle, tableName)); + return connector.retryWithHandle( + handle -> connector.tableExists(handle, tableName.toUpperCase(Locale.ENGLISH)) + ); } } diff --git a/services/pom.xml b/services/pom.xml index 6f1813745aab..42dadd57aa24 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -222,6 +222,11 @@ test-jar test + + org.jdbi + jdbi + test + junit junit From 057252e64d44d10a80e030818315823d933ec685 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 24 Sep 2023 21:55:28 +0530 Subject: [PATCH 42/43] Fix spotted bugs --- .../common/task/AbstractBatchIndexTask.java | 26 +++++++++++++++++++ .../AppenderatorDriverRealtimeIndexTask.java | 5 ++-- .../druid/indexing/common/task/IndexTask.java | 20 +++----------- .../parallel/ParallelIndexSupervisorTask.java | 23 +++------------- .../IndexerSQLMetadataStorageCoordinator.java | 11 ++++---- 5 files changed, 41 insertions(+), 44 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index c88d2274ca05..ea61f37c7e90 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -36,6 +36,10 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; +import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; @@ -43,6 +47,7 @@ import org.apache.druid.indexing.common.task.batch.MaxAllowedLocksExceededException; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.indexing.input.InputRowSchemas; +import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -394,6 +399,27 @@ private boolean tryLockWithDetermineResult(TaskActionClient client, LockGranular } } + /** + * Builds a TaskAction to publish segments based on the type of locks that this + * task acquires (determined by context property {@link Tasks#TASK_LOCK_TYPE}). + */ + protected TaskAction buildPublishAction( + Set segmentsToBeOverwritten, + Set segmentsToPublish + ) + { + TaskLockType lockType = TaskLockType.valueOf( + getContextValue(Tasks.TASK_LOCK_TYPE, Tasks.DEFAULT_TASK_LOCK_TYPE.name()) + ); + switch (lockType) { + case REPLACE: + return SegmentTransactionalReplaceAction.create(segmentsToPublish); + case APPEND: + return SegmentTransactionalAppendAction.create(segmentsToPublish); + default: + return SegmentTransactionalInsertAction.overwriteAction(segmentsToBeOverwritten, segmentsToPublish); + } + } protected boolean tryTimeChunkLock(TaskActionClient client, List intervals) throws IOException { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index e66a28793b6a..dfa1f85fde72 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -55,6 +55,7 @@ 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; @@ -293,9 +294,7 @@ public TaskStatus runTask(final TaskToolbox toolbox) DiscoveryDruidNode discoveryDruidNode = createDiscoveryDruidNode(toolbox); appenderator = newAppenderator(dataSchema, tuningConfig, metrics, toolbox); - final TaskLockType lockType = TaskLockType.valueOf( - getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name()) - ); + final TaskLockType lockType = TaskLocks.determineLockTypeForAppend(getContext()); StreamAppenderatorDriver driver = newDriver(dataSchema, appenderator, toolbox, metrics, lockType); try { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index beabb9490a7d..73d4bfeb8bf7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -915,23 +915,9 @@ private TaskStatus generateAndPublishSegments( } - final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> { - TaskLockType lockType = TaskLockType.valueOf(getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name())); - switch (lockType) { - case REPLACE: - return toolbox.getTaskActionClient().submit( - SegmentTransactionalReplaceAction.create(segmentsToPublish) - ); - case APPEND: - return toolbox.getTaskActionClient().submit( - SegmentTransactionalAppendAction.create(segmentsToPublish) - ); - default: - return toolbox.getTaskActionClient().submit( - SegmentTransactionalInsertAction.overwriteAction(segmentsToBeOverwritten, segmentsToPublish) - ); - } - }; + final TransactionalSegmentPublisher publisher = + (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> + toolbox.getTaskActionClient().submit(buildPublishAction(segmentsToBeOverwritten, segmentsToPublish)); String effectiveId = getContextValue(CompactionTask.CTX_KEY_APPENDERATOR_TRACKING_TASK_ID, null); if (effectiveId == null) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 179f31614061..a50b550fde4d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -1171,25 +1171,10 @@ private void publishSegments( } } - final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> { - final TaskLockType lockType = TaskLockType.valueOf( - getContextValue(Tasks.TASK_LOCK_TYPE, TaskLockType.EXCLUSIVE.name()) - ); - switch (lockType) { - case REPLACE: - return toolbox.getTaskActionClient().submit( - SegmentTransactionalReplaceAction.create(segmentsToPublish) - ); - case APPEND: - return toolbox.getTaskActionClient().submit( - SegmentTransactionalAppendAction.create(segmentsToPublish) - ); - default: - return toolbox.getTaskActionClient().submit( - SegmentTransactionalInsertAction.overwriteAction(segmentsToBeOverwritten, segmentsToPublish) - ); - } - }; + final TransactionalSegmentPublisher publisher = + (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> + toolbox.getTaskActionClient().submit(buildPublishAction(segmentsToBeOverwritten, segmentsToPublish)); + final boolean published = newSegments.isEmpty() || publisher.publishSegments(oldSegments, newSegments, annotateFunction, null).isSuccess(); diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 9dfb8b7db68d..7eaac692f7ce 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1089,18 +1089,19 @@ private Set getSegmentsToUpgradeOnAppend( } final Set upgradedSegments = new HashSet<>(); - for (String upgradeVersion : committedVersionToIntervals.keySet()) { + for (Map.Entry> entry : committedVersionToIntervals.entrySet()) { + final String upgradeVersion = entry.getKey(); Map> segmentsToUpgrade = getSegmentsWithVersionLowerThan( upgradeVersion, - committedVersionToIntervals.get(upgradeVersion), + entry.getValue(), appendVersionToSegments ); - for (Map.Entry> entry : segmentsToUpgrade.entrySet()) { + for (Map.Entry> upgradeEntry : segmentsToUpgrade.entrySet()) { Set segmentsUpgradedToVersion = upgradeSegmentsToVersion( handle, upgradeVersion, - entry.getKey(), - entry.getValue(), + upgradeEntry.getKey(), + upgradeEntry.getValue(), committedIntervalToSegments ); log.info("Upgraded [%d] segments to version[%s].", segmentsUpgradedToVersion.size(), upgradeVersion); From 7df51f26b9b8b2dd2165fa3d9d606efafab7f853 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 24 Sep 2023 22:42:05 +0530 Subject: [PATCH 43/43] Fix checkstyle --- .../java/org/apache/druid/indexing/common/task/IndexTask.java | 4 ---- .../task/batch/parallel/ParallelIndexSupervisorTask.java | 4 ---- 2 files changed, 8 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 73d4bfeb8bf7..a2ca4f869ea7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -50,13 +50,9 @@ import org.apache.druid.indexer.partitions.SecondaryPartitionType; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; -import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; -import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; -import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentGenerateTask; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index a50b550fde4d..d3e218623cda 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -42,12 +42,8 @@ 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.TaskLockType; import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.SegmentTransactionalAppendAction; -import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; -import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.CurrentSubTaskHolder;