kfaraz commented on code in PR #16755:
URL: https://github.com/apache/druid/pull/16755#discussion_r1702487656


##########
integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITConcurrentAppendReplaceTest.java:
##########
@@ -0,0 +1,541 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.tests.coordinator.duty;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+import org.apache.druid.indexer.partitions.PartitionsSpec;
+import org.apache.druid.indexing.common.TaskLock;
+import org.apache.druid.indexing.common.TaskLockType;
+import org.apache.druid.indexing.common.task.Tasks;
+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.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.metadata.LockFilterPolicy;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
+import org.apache.druid.server.coordinator.DruidCompactionConfig;
+import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig;
+import org.apache.druid.testing.clients.CompactionResourceTestClient;
+import org.apache.druid.testing.clients.TaskResponseObject;
+import org.apache.druid.testing.guice.DruidTestModuleFactory;
+import org.apache.druid.testing.utils.EventSerializer;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.KafkaUtil;
+import org.apache.druid.testing.utils.MsqTestQueryHelper;
+import org.apache.druid.testing.utils.StreamEventWriter;
+import org.apache.druid.testing.utils.StreamGenerator;
+import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator;
+import org.apache.druid.tests.TestNGGroup;
+import org.apache.druid.tests.indexer.AbstractKafkaIndexingServiceTest;
+import org.apache.druid.tests.indexer.AbstractStreamIndexingTest;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.utils.CollectionUtils;
+import org.joda.time.Interval;
+import org.joda.time.Period;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+
+/**
+ * Integration Test to verify behaviour when there are concurrent
+ * compaction tasks with ongoing stream ingestion tasks.
+ */
+@Test(groups = {TestNGGroup.COMPACTION})
+@Guice(moduleFactory = DruidTestModuleFactory.class)
+public class ITConcurrentAppendReplaceTest extends 
AbstractKafkaIndexingServiceTest
+{
+  private static final Logger LOG = new 
Logger(ITConcurrentAppendReplaceTest.class);
+
+  @Inject
+  private CompactionResourceTestClient compactionResource;
+  
+  @Inject
+  private MsqTestQueryHelper msqHelper;
+
+  private GeneratedTestConfig generatedTestConfig;
+  private StreamGenerator streamGenerator;
+
+  private String fullDatasourceName;
+
+  private int currentRowCount = 0;
+  private boolean concurrentAppendAndReplaceLocksExisted;
+
+  @DataProvider
+  public static Object[] getParameters()
+  {
+    return new Object[]{false, true};
+  }
+
+  @BeforeClass
+  public void setupClass() throws Exception
+  {
+    doBeforeClass();
+  }
+
+  @BeforeMethod
+  public void setup() throws Exception
+  {
+    generatedTestConfig = new GeneratedTestConfig(
+        Specs.PARSER_TYPE,
+        getResourceAsString(Specs.INPUT_FORMAT_PATH)
+    );
+    fullDatasourceName = generatedTestConfig.getFullDatasourceName();
+    final EventSerializer serializer = jsonMapper.readValue(
+        getResourceAsStream(Specs.SERIALIZER_PATH),
+        EventSerializer.class
+    );
+    streamGenerator = new WikipediaStreamEventStreamGenerator(serializer, 6, 
100);
+    concurrentAppendAndReplaceLocksExisted = false;
+  }
+
+  @Override
+  public String getTestNamePrefix()
+  {
+    return "autocompact_lock_contention";
+  }
+
+  @Test(dataProvider = "getParameters")
+  public void testConcurrentStreamAppendReplace(boolean transactionEnabled) 
throws Exception
+  {
+    if (shouldSkipTest(transactionEnabled)) {
+      return;
+    }
+
+    try (
+        final Closeable closer = createResourceCloser(generatedTestConfig);
+        final StreamEventWriter streamEventWriter = 
createStreamEventWriter(config, transactionEnabled)
+    ) {
+      // Start supervisor
+      final String taskSpec
+          = generatedTestConfig.getStreamIngestionPropsTransform()
+                               
.apply(getResourceAsString(SUPERVISOR_WITH_CONCURRENT_LOCKS_SPEC_TEMPLATE_PATH));
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("supervisorSpec: [%s]", taskSpec);
+
+      // Generate data for minutes 1, 2 and 3
+      final Interval minute1 = 
Intervals.of("2000-01-01T01:01:00Z/2000-01-01T01:02:00Z");
+      long rowsForMinute1 = generateData(minute1, streamEventWriter);
+
+      final Interval minute2 = 
Intervals.of("2000-01-01T01:02:00Z/2000-01-01T01:03:00Z");
+      long rowsForMinute2 = generateData(minute2, streamEventWriter);
+
+      final Interval minute3 = 
Intervals.of("2000-01-01T01:03:00Z/2000-01-01T01:04:00Z");
+      long rowsForMinute3 = generateData(minute3, streamEventWriter);
+
+      Function<String, AggregatorFactory> function = name -> new 
LongSumAggregatorFactory(name, "count");
+
+      // Wait for data to be ingested for all the minutes
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+
+      ensureSegmentsLoaded(fullDatasourceName);
+
+      // 2 segments for each minute, total 6
+      ensureSegmentsCount(fullDatasourceName, 6);
+      printTaskStatuses(fullDatasourceName);

Review Comment:
   Nit: If the segment count is already satisfied before this, we do we need to 
print the task statuses?



##########
integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITConcurrentAppendReplaceTest.java:
##########
@@ -0,0 +1,541 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.tests.coordinator.duty;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+import org.apache.druid.indexer.partitions.PartitionsSpec;
+import org.apache.druid.indexing.common.TaskLock;
+import org.apache.druid.indexing.common.TaskLockType;
+import org.apache.druid.indexing.common.task.Tasks;
+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.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.metadata.LockFilterPolicy;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
+import org.apache.druid.server.coordinator.DruidCompactionConfig;
+import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig;
+import org.apache.druid.testing.clients.CompactionResourceTestClient;
+import org.apache.druid.testing.clients.TaskResponseObject;
+import org.apache.druid.testing.guice.DruidTestModuleFactory;
+import org.apache.druid.testing.utils.EventSerializer;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.KafkaUtil;
+import org.apache.druid.testing.utils.MsqTestQueryHelper;
+import org.apache.druid.testing.utils.StreamEventWriter;
+import org.apache.druid.testing.utils.StreamGenerator;
+import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator;
+import org.apache.druid.tests.TestNGGroup;
+import org.apache.druid.tests.indexer.AbstractKafkaIndexingServiceTest;
+import org.apache.druid.tests.indexer.AbstractStreamIndexingTest;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.utils.CollectionUtils;
+import org.joda.time.Interval;
+import org.joda.time.Period;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+
+/**
+ * Integration Test to verify behaviour when there are concurrent
+ * compaction tasks with ongoing stream ingestion tasks.
+ */
+@Test(groups = {TestNGGroup.COMPACTION})
+@Guice(moduleFactory = DruidTestModuleFactory.class)
+public class ITConcurrentAppendReplaceTest extends 
AbstractKafkaIndexingServiceTest
+{
+  private static final Logger LOG = new 
Logger(ITConcurrentAppendReplaceTest.class);
+
+  @Inject
+  private CompactionResourceTestClient compactionResource;
+  
+  @Inject
+  private MsqTestQueryHelper msqHelper;
+
+  private GeneratedTestConfig generatedTestConfig;
+  private StreamGenerator streamGenerator;
+
+  private String fullDatasourceName;
+
+  private int currentRowCount = 0;
+  private boolean concurrentAppendAndReplaceLocksExisted;
+
+  @DataProvider
+  public static Object[] getParameters()
+  {
+    return new Object[]{false, true};
+  }
+
+  @BeforeClass
+  public void setupClass() throws Exception
+  {
+    doBeforeClass();
+  }
+
+  @BeforeMethod
+  public void setup() throws Exception
+  {
+    generatedTestConfig = new GeneratedTestConfig(
+        Specs.PARSER_TYPE,
+        getResourceAsString(Specs.INPUT_FORMAT_PATH)
+    );
+    fullDatasourceName = generatedTestConfig.getFullDatasourceName();
+    final EventSerializer serializer = jsonMapper.readValue(
+        getResourceAsStream(Specs.SERIALIZER_PATH),
+        EventSerializer.class
+    );
+    streamGenerator = new WikipediaStreamEventStreamGenerator(serializer, 6, 
100);
+    concurrentAppendAndReplaceLocksExisted = false;
+  }
+
+  @Override
+  public String getTestNamePrefix()
+  {
+    return "autocompact_lock_contention";
+  }
+
+  @Test(dataProvider = "getParameters")
+  public void testConcurrentStreamAppendReplace(boolean transactionEnabled) 
throws Exception
+  {
+    if (shouldSkipTest(transactionEnabled)) {
+      return;
+    }
+
+    try (
+        final Closeable closer = createResourceCloser(generatedTestConfig);
+        final StreamEventWriter streamEventWriter = 
createStreamEventWriter(config, transactionEnabled)
+    ) {
+      // Start supervisor
+      final String taskSpec
+          = generatedTestConfig.getStreamIngestionPropsTransform()
+                               
.apply(getResourceAsString(SUPERVISOR_WITH_CONCURRENT_LOCKS_SPEC_TEMPLATE_PATH));
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("supervisorSpec: [%s]", taskSpec);
+
+      // Generate data for minutes 1, 2 and 3
+      final Interval minute1 = 
Intervals.of("2000-01-01T01:01:00Z/2000-01-01T01:02:00Z");
+      long rowsForMinute1 = generateData(minute1, streamEventWriter);
+
+      final Interval minute2 = 
Intervals.of("2000-01-01T01:02:00Z/2000-01-01T01:03:00Z");
+      long rowsForMinute2 = generateData(minute2, streamEventWriter);
+
+      final Interval minute3 = 
Intervals.of("2000-01-01T01:03:00Z/2000-01-01T01:04:00Z");
+      long rowsForMinute3 = generateData(minute3, streamEventWriter);
+
+      Function<String, AggregatorFactory> function = name -> new 
LongSumAggregatorFactory(name, "count");
+
+      // Wait for data to be ingested for all the minutes
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+
+      ensureSegmentsLoaded(fullDatasourceName);
+
+      // 2 segments for each minute, total 6
+      ensureSegmentsCount(fullDatasourceName, 6);
+      printTaskStatuses(fullDatasourceName);
+
+      // Trigger auto compaction
+      submitAndVerifyCompactionConfig(fullDatasourceName, null);
+      compactionResource.forceTriggerAutoCompaction();
+
+      // Verify that all the segments are now compacted
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+      ensureSegmentsLoaded(fullDatasourceName);
+      ensureSegmentsCount(fullDatasourceName, 3);
+      verifyCompactedIntervals(fullDatasourceName, minute1, minute2, minute3);
+      printTaskStatuses(fullDatasourceName);
+
+      // Concurrent compaction with configured segment granularity
+      for (int i = 0; i < 5; i++) {
+        rowsForMinute1 += generateData(minute1, streamEventWriter);
+        rowsForMinute2 += generateData(minute2, streamEventWriter);
+        rowsForMinute3 += generateData(minute3, streamEventWriter);
+
+        //ensureNoLockContention();
+        compactionResource.forceTriggerAutoCompaction();
+        ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+        printTaskStatuses(fullDatasourceName);
+        checkAndSetConcurrentLocks(fullDatasourceName);
+      }
+
+      // Verify the state with minute granularity
+      ensureSegmentsCount(fullDatasourceName, 3);
+      ensureSegmentsLoaded(fullDatasourceName);
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+      verifyCompactedIntervals(fullDatasourceName, minute1, minute2, minute3);
+
+
+      // Use ALL segment granularity for compaction and run concurrent 
streaming ingestion
+      submitAndVerifyCompactionConfig(fullDatasourceName, Granularities.DAY);
+
+      for (int i = 0; i < 5; i++) {
+        rowsForMinute1 += generateData(minute1, streamEventWriter);
+        rowsForMinute2 += generateData(minute2, streamEventWriter);
+        rowsForMinute3 += generateData(minute3, streamEventWriter);
+
+        //ensureNoLockContention();
+        compactionResource.forceTriggerAutoCompaction();
+        ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+        printTaskStatuses(fullDatasourceName);
+        checkAndSetConcurrentLocks(fullDatasourceName);
+      }
+
+      // Verify the state with all granularity
+      ensureSegmentsCount(fullDatasourceName, 1);
+      ensureSegmentsLoaded(fullDatasourceName);
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+      verifyCompactedIntervals(fullDatasourceName, 
Intervals.of("2000-01-01/2000-01-02"));
+      printTaskStatuses(fullDatasourceName);
+
+      Assert.assertTrue(concurrentAppendAndReplaceLocksExisted);
+    }
+  }
+
+
+  @Test(dataProvider = "getParameters")
+  public void testConcurrentMSQAppendReplace(boolean transactionEnabled) 
throws Exception
+  {
+    if (shouldSkipTest(transactionEnabled)) {
+      return;
+    }
+
+    final String datasource = "dst";
+
+    final String queryLocal =
+        StringUtils.format(
+            "INSERT INTO %s\n"

Review Comment:
   See if it is possible to use a short and simple query to make the test 
assertions. If not, put this query in a static `.sql` file.



##########
integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITConcurrentAppendReplaceTest.java:
##########
@@ -0,0 +1,541 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.tests.coordinator.duty;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+import org.apache.druid.indexer.partitions.PartitionsSpec;
+import org.apache.druid.indexing.common.TaskLock;
+import org.apache.druid.indexing.common.TaskLockType;
+import org.apache.druid.indexing.common.task.Tasks;
+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.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.metadata.LockFilterPolicy;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
+import org.apache.druid.server.coordinator.DruidCompactionConfig;
+import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig;
+import org.apache.druid.testing.clients.CompactionResourceTestClient;
+import org.apache.druid.testing.clients.TaskResponseObject;
+import org.apache.druid.testing.guice.DruidTestModuleFactory;
+import org.apache.druid.testing.utils.EventSerializer;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.KafkaUtil;
+import org.apache.druid.testing.utils.MsqTestQueryHelper;
+import org.apache.druid.testing.utils.StreamEventWriter;
+import org.apache.druid.testing.utils.StreamGenerator;
+import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator;
+import org.apache.druid.tests.TestNGGroup;
+import org.apache.druid.tests.indexer.AbstractKafkaIndexingServiceTest;
+import org.apache.druid.tests.indexer.AbstractStreamIndexingTest;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.utils.CollectionUtils;
+import org.joda.time.Interval;
+import org.joda.time.Period;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+
+/**
+ * Integration Test to verify behaviour when there are concurrent
+ * compaction tasks with ongoing stream ingestion tasks.
+ */
+@Test(groups = {TestNGGroup.COMPACTION})
+@Guice(moduleFactory = DruidTestModuleFactory.class)
+public class ITConcurrentAppendReplaceTest extends 
AbstractKafkaIndexingServiceTest
+{
+  private static final Logger LOG = new 
Logger(ITConcurrentAppendReplaceTest.class);
+
+  @Inject
+  private CompactionResourceTestClient compactionResource;
+  
+  @Inject
+  private MsqTestQueryHelper msqHelper;
+
+  private GeneratedTestConfig generatedTestConfig;
+  private StreamGenerator streamGenerator;
+
+  private String fullDatasourceName;
+
+  private int currentRowCount = 0;
+  private boolean concurrentAppendAndReplaceLocksExisted;
+
+  @DataProvider
+  public static Object[] getParameters()
+  {
+    return new Object[]{false, true};
+  }
+
+  @BeforeClass
+  public void setupClass() throws Exception
+  {
+    doBeforeClass();
+  }
+
+  @BeforeMethod
+  public void setup() throws Exception
+  {
+    generatedTestConfig = new GeneratedTestConfig(
+        Specs.PARSER_TYPE,
+        getResourceAsString(Specs.INPUT_FORMAT_PATH)
+    );
+    fullDatasourceName = generatedTestConfig.getFullDatasourceName();
+    final EventSerializer serializer = jsonMapper.readValue(
+        getResourceAsStream(Specs.SERIALIZER_PATH),
+        EventSerializer.class
+    );
+    streamGenerator = new WikipediaStreamEventStreamGenerator(serializer, 6, 
100);
+    concurrentAppendAndReplaceLocksExisted = false;
+  }
+
+  @Override
+  public String getTestNamePrefix()
+  {
+    return "autocompact_lock_contention";
+  }
+
+  @Test(dataProvider = "getParameters")
+  public void testConcurrentStreamAppendReplace(boolean transactionEnabled) 
throws Exception
+  {
+    if (shouldSkipTest(transactionEnabled)) {
+      return;
+    }
+
+    try (
+        final Closeable closer = createResourceCloser(generatedTestConfig);
+        final StreamEventWriter streamEventWriter = 
createStreamEventWriter(config, transactionEnabled)
+    ) {
+      // Start supervisor
+      final String taskSpec
+          = generatedTestConfig.getStreamIngestionPropsTransform()
+                               
.apply(getResourceAsString(SUPERVISOR_WITH_CONCURRENT_LOCKS_SPEC_TEMPLATE_PATH));
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("supervisorSpec: [%s]", taskSpec);
+
+      // Generate data for minutes 1, 2 and 3
+      final Interval minute1 = 
Intervals.of("2000-01-01T01:01:00Z/2000-01-01T01:02:00Z");
+      long rowsForMinute1 = generateData(minute1, streamEventWriter);
+
+      final Interval minute2 = 
Intervals.of("2000-01-01T01:02:00Z/2000-01-01T01:03:00Z");
+      long rowsForMinute2 = generateData(minute2, streamEventWriter);
+
+      final Interval minute3 = 
Intervals.of("2000-01-01T01:03:00Z/2000-01-01T01:04:00Z");
+      long rowsForMinute3 = generateData(minute3, streamEventWriter);
+
+      Function<String, AggregatorFactory> function = name -> new 
LongSumAggregatorFactory(name, "count");
+
+      // Wait for data to be ingested for all the minutes
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+
+      ensureSegmentsLoaded(fullDatasourceName);
+
+      // 2 segments for each minute, total 6
+      ensureSegmentsCount(fullDatasourceName, 6);
+      printTaskStatuses(fullDatasourceName);
+
+      // Trigger auto compaction
+      submitAndVerifyCompactionConfig(fullDatasourceName, null);
+      compactionResource.forceTriggerAutoCompaction();
+
+      // Verify that all the segments are now compacted
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+      ensureSegmentsLoaded(fullDatasourceName);
+      ensureSegmentsCount(fullDatasourceName, 3);
+      verifyCompactedIntervals(fullDatasourceName, minute1, minute2, minute3);
+      printTaskStatuses(fullDatasourceName);
+
+      // Concurrent compaction with configured segment granularity
+      for (int i = 0; i < 5; i++) {
+        rowsForMinute1 += generateData(minute1, streamEventWriter);
+        rowsForMinute2 += generateData(minute2, streamEventWriter);
+        rowsForMinute3 += generateData(minute3, streamEventWriter);
+
+        //ensureNoLockContention();

Review Comment:
   Please remove this line if not needed.



##########
integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITConcurrentAppendReplaceTest.java:
##########
@@ -0,0 +1,541 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.tests.coordinator.duty;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+import org.apache.druid.indexer.partitions.PartitionsSpec;
+import org.apache.druid.indexing.common.TaskLock;
+import org.apache.druid.indexing.common.TaskLockType;
+import org.apache.druid.indexing.common.task.Tasks;
+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.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.metadata.LockFilterPolicy;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
+import org.apache.druid.server.coordinator.DruidCompactionConfig;
+import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig;
+import org.apache.druid.testing.clients.CompactionResourceTestClient;
+import org.apache.druid.testing.clients.TaskResponseObject;
+import org.apache.druid.testing.guice.DruidTestModuleFactory;
+import org.apache.druid.testing.utils.EventSerializer;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.KafkaUtil;
+import org.apache.druid.testing.utils.MsqTestQueryHelper;
+import org.apache.druid.testing.utils.StreamEventWriter;
+import org.apache.druid.testing.utils.StreamGenerator;
+import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator;
+import org.apache.druid.tests.TestNGGroup;
+import org.apache.druid.tests.indexer.AbstractKafkaIndexingServiceTest;
+import org.apache.druid.tests.indexer.AbstractStreamIndexingTest;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.utils.CollectionUtils;
+import org.joda.time.Interval;
+import org.joda.time.Period;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+
+/**
+ * Integration Test to verify behaviour when there are concurrent
+ * compaction tasks with ongoing stream ingestion tasks.
+ */
+@Test(groups = {TestNGGroup.COMPACTION})
+@Guice(moduleFactory = DruidTestModuleFactory.class)
+public class ITConcurrentAppendReplaceTest extends 
AbstractKafkaIndexingServiceTest
+{
+  private static final Logger LOG = new 
Logger(ITConcurrentAppendReplaceTest.class);
+
+  @Inject
+  private CompactionResourceTestClient compactionResource;
+  
+  @Inject
+  private MsqTestQueryHelper msqHelper;
+
+  private GeneratedTestConfig generatedTestConfig;
+  private StreamGenerator streamGenerator;
+
+  private String fullDatasourceName;
+
+  private int currentRowCount = 0;
+  private boolean concurrentAppendAndReplaceLocksExisted;
+
+  @DataProvider
+  public static Object[] getParameters()
+  {
+    return new Object[]{false, true};
+  }
+
+  @BeforeClass
+  public void setupClass() throws Exception
+  {
+    doBeforeClass();
+  }
+
+  @BeforeMethod
+  public void setup() throws Exception
+  {
+    generatedTestConfig = new GeneratedTestConfig(
+        Specs.PARSER_TYPE,
+        getResourceAsString(Specs.INPUT_FORMAT_PATH)
+    );
+    fullDatasourceName = generatedTestConfig.getFullDatasourceName();
+    final EventSerializer serializer = jsonMapper.readValue(
+        getResourceAsStream(Specs.SERIALIZER_PATH),
+        EventSerializer.class
+    );
+    streamGenerator = new WikipediaStreamEventStreamGenerator(serializer, 6, 
100);
+    concurrentAppendAndReplaceLocksExisted = false;
+  }
+
+  @Override
+  public String getTestNamePrefix()
+  {
+    return "autocompact_lock_contention";
+  }
+
+  @Test(dataProvider = "getParameters")
+  public void testConcurrentStreamAppendReplace(boolean transactionEnabled) 
throws Exception
+  {
+    if (shouldSkipTest(transactionEnabled)) {
+      return;
+    }
+
+    try (
+        final Closeable closer = createResourceCloser(generatedTestConfig);
+        final StreamEventWriter streamEventWriter = 
createStreamEventWriter(config, transactionEnabled)
+    ) {
+      // Start supervisor
+      final String taskSpec
+          = generatedTestConfig.getStreamIngestionPropsTransform()
+                               
.apply(getResourceAsString(SUPERVISOR_WITH_CONCURRENT_LOCKS_SPEC_TEMPLATE_PATH));
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("supervisorSpec: [%s]", taskSpec);
+
+      // Generate data for minutes 1, 2 and 3
+      final Interval minute1 = 
Intervals.of("2000-01-01T01:01:00Z/2000-01-01T01:02:00Z");
+      long rowsForMinute1 = generateData(minute1, streamEventWriter);
+
+      final Interval minute2 = 
Intervals.of("2000-01-01T01:02:00Z/2000-01-01T01:03:00Z");
+      long rowsForMinute2 = generateData(minute2, streamEventWriter);
+
+      final Interval minute3 = 
Intervals.of("2000-01-01T01:03:00Z/2000-01-01T01:04:00Z");
+      long rowsForMinute3 = generateData(minute3, streamEventWriter);
+
+      Function<String, AggregatorFactory> function = name -> new 
LongSumAggregatorFactory(name, "count");
+
+      // Wait for data to be ingested for all the minutes
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+
+      ensureSegmentsLoaded(fullDatasourceName);
+
+      // 2 segments for each minute, total 6
+      ensureSegmentsCount(fullDatasourceName, 6);
+      printTaskStatuses(fullDatasourceName);
+
+      // Trigger auto compaction
+      submitAndVerifyCompactionConfig(fullDatasourceName, null);
+      compactionResource.forceTriggerAutoCompaction();
+
+      // Verify that all the segments are now compacted
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+      ensureSegmentsLoaded(fullDatasourceName);
+      ensureSegmentsCount(fullDatasourceName, 3);
+      verifyCompactedIntervals(fullDatasourceName, minute1, minute2, minute3);
+      printTaskStatuses(fullDatasourceName);
+
+      // Concurrent compaction with configured segment granularity
+      for (int i = 0; i < 5; i++) {
+        rowsForMinute1 += generateData(minute1, streamEventWriter);
+        rowsForMinute2 += generateData(minute2, streamEventWriter);
+        rowsForMinute3 += generateData(minute3, streamEventWriter);
+
+        //ensureNoLockContention();
+        compactionResource.forceTriggerAutoCompaction();
+        ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+        printTaskStatuses(fullDatasourceName);
+        checkAndSetConcurrentLocks(fullDatasourceName);
+      }
+
+      // Verify the state with minute granularity
+      ensureSegmentsCount(fullDatasourceName, 3);
+      ensureSegmentsLoaded(fullDatasourceName);
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+      verifyCompactedIntervals(fullDatasourceName, minute1, minute2, minute3);
+
+
+      // Use ALL segment granularity for compaction and run concurrent 
streaming ingestion
+      submitAndVerifyCompactionConfig(fullDatasourceName, Granularities.DAY);
+
+      for (int i = 0; i < 5; i++) {
+        rowsForMinute1 += generateData(minute1, streamEventWriter);
+        rowsForMinute2 += generateData(minute2, streamEventWriter);
+        rowsForMinute3 += generateData(minute3, streamEventWriter);
+
+        //ensureNoLockContention();
+        compactionResource.forceTriggerAutoCompaction();
+        ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+        printTaskStatuses(fullDatasourceName);
+        checkAndSetConcurrentLocks(fullDatasourceName);
+      }
+
+      // Verify the state with all granularity
+      ensureSegmentsCount(fullDatasourceName, 1);
+      ensureSegmentsLoaded(fullDatasourceName);
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+      verifyCompactedIntervals(fullDatasourceName, 
Intervals.of("2000-01-01/2000-01-02"));
+      printTaskStatuses(fullDatasourceName);
+
+      Assert.assertTrue(concurrentAppendAndReplaceLocksExisted);
+    }
+  }
+
+
+  @Test(dataProvider = "getParameters")
+  public void testConcurrentMSQAppendReplace(boolean transactionEnabled) 
throws Exception
+  {
+    if (shouldSkipTest(transactionEnabled)) {
+      return;
+    }
+
+    final String datasource = "dst";
+
+    final String queryLocal =
+        StringUtils.format(
+            "INSERT INTO %s\n"
+            + "SELECT\n"
+            + "  TIME_PARSE(\"timestamp\") AS __time,\n"
+            + "  isRobot,\n"
+            + "  diffUrl,\n"
+            + "  added,\n"
+            + "  countryIsoCode,\n"
+            + "  regionName,\n"
+            + "  channel,\n"
+            + "  flags,\n"
+            + "  delta,\n"
+            + "  isUnpatrolled,\n"
+            + "  isNew,\n"
+            + "  deltaBucket,\n"
+            + "  isMinor,\n"
+            + "  isAnonymous,\n"
+            + "  deleted,\n"
+            + "  cityName,\n"
+            + "  metroCode,\n"
+            + "  namespace,\n"
+            + "  comment,\n"
+            + "  page,\n"
+            + "  commentLength,\n"
+            + "  countryName,\n"
+            + "  user,\n"
+            + "  regionIsoCode\n"
+            + "FROM TABLE(\n"
+            + "  EXTERN(\n"
+            + "    
'{\"type\":\"local\",\"files\":[\"/resources/data/batch_index/json/wikipedia_index_data1.json\"]}',\n"
+            + "    '{\"type\":\"json\"}',\n"
+            + "    
'[{\"type\":\"string\",\"name\":\"timestamp\"},{\"type\":\"string\",\"name\":\"isRobot\"},{\"type\":\"string\",\"name\":\"diffUrl\"},{\"type\":\"long\",\"name\":\"added\"},{\"type\":\"string\",\"name\":\"countryIsoCode\"},{\"type\":\"string\",\"name\":\"regionName\"},{\"type\":\"string\",\"name\":\"channel\"},{\"type\":\"string\",\"name\":\"flags\"},{\"type\":\"long\",\"name\":\"delta\"},{\"type\":\"string\",\"name\":\"isUnpatrolled\"},{\"type\":\"string\",\"name\":\"isNew\"},{\"type\":\"double\",\"name\":\"deltaBucket\"},{\"type\":\"string\",\"name\":\"isMinor\"},{\"type\":\"string\",\"name\":\"isAnonymous\"},{\"type\":\"long\",\"name\":\"deleted\"},{\"type\":\"string\",\"name\":\"cityName\"},{\"type\":\"long\",\"name\":\"metroCode\"},{\"type\":\"string\",\"name\":\"namespace\"},{\"type\":\"string\",\"name\":\"comment\"},{\"type\":\"string\",\"name\":\"page\"},{\"type\":\"long\",\"name\":\"commentLength\"},{\"type\":\"string\",\"name\":\"countryName\"},{\"type\":
 
\"string\",\"name\":\"user\"},{\"type\":\"string\",\"name\":\"regionIsoCode\"}]'\n"
+            + "  )\n"
+            + ")\n"
+            + "PARTITIONED BY DAY\n",
+            datasource
+        );
+
+    Function<String, AggregatorFactory> function = name -> new 
CountAggregatorFactory("rows");
+
+    msqHelper.submitMsqTaskSuccesfully(queryLocal, 
ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true));
+    ensureSegmentsCount(datasource, 1);
+    ensureRowCount(datasource, 3, function);
+
+    submitAndVerifyCompactionConfig(datasource, null);
+
+    for (int i = 0; i < 5; i++) {
+      // Submit the task and wait for the datasource to get loaded
+      msqHelper.submitMsqTaskSuccesfully(queryLocal, 
ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true));
+
+      ensureRowCount(datasource, (i + 2) * 3, function);
+    }
+
+    ensureRowCount(datasource, 18, function);
+    ensureSegmentsCount(datasource, 1);
+    ensureSegmentsLoaded(datasource);
+    verifyCompactedIntervals(datasource, 
Intervals.of("2013-08-31/2013-09-01"));
+    printTaskStatuses(datasource);
+
+    Assert.assertTrue(concurrentAppendAndReplaceLocksExisted);
+  }
+
+
+    /**
+     * Retries until the segment count is as expected.
+     */
+  private void ensureSegmentsCount(String datasource, int numExpectedSegments)
+  {
+    ITRetryUtil.retryUntilTrue(
+        () -> {
+          printTaskStatuses(datasource);
+          checkAndSetConcurrentLocks(datasource);
+          compactionResource.forceTriggerAutoCompaction();
+          List<DataSegment> segments = 
coordinator.getFullSegmentsMetadata(datasource);
+          StringBuilder sb = new StringBuilder();
+          segments.forEach(
+              seg -> sb.append("{")
+                       .append(seg.getId())
+                       .append(", ")
+                       .append(seg.getSize())
+                       .append("}, ")
+          );
+          LOG.info("Found Segments: %s", sb);
+          LOG.info("Current metadata segment count: %d, expected: %d", 
segments.size(), numExpectedSegments);
+          return segments.size() == numExpectedSegments;
+        },
+        "Segment count check"
+    );
+  }
+
+  /**
+   * Verifies that the given intervals have been compacted.
+   */
+  private void verifyCompactedIntervals(String datasource, Interval... 
compactedIntervals)
+  {
+    List<DataSegment> segments = 
coordinator.getFullSegmentsMetadata(datasource);
+    List<DataSegment> observedCompactedSegments = new ArrayList<>();
+    Set<Interval> observedCompactedIntervals = new HashSet<>();
+    for (DataSegment segment : segments) {
+      if (segment.getLastCompactionState() != null) {
+        observedCompactedSegments.add(segment);
+        observedCompactedIntervals.add(segment.getInterval());
+      }
+    }
+
+    Set<Interval> expectedCompactedIntervals = new 
HashSet<>(Arrays.asList(compactedIntervals));
+    Assert.assertEquals(observedCompactedIntervals, 
expectedCompactedIntervals);
+
+    DynamicPartitionsSpec expectedPartitionSpec = new DynamicPartitionsSpec(
+        PartitionsSpec.DEFAULT_MAX_ROWS_PER_SEGMENT,
+        Long.MAX_VALUE
+    );
+    for (DataSegment compactedSegment : observedCompactedSegments) {
+      Assert.assertNotNull(compactedSegment.getLastCompactionState());
+      Assert.assertEquals(
+          compactedSegment.getLastCompactionState().getPartitionsSpec(),
+          expectedPartitionSpec
+      );
+    }
+  }
+
+  /**
+   * Generates data points for the specified interval.
+   *
+   * @return Number of rows generated.
+   */
+  private long generateData(Interval interval, StreamEventWriter 
streamEventWriter)
+  {
+    long rowCount = streamGenerator.run(
+        generatedTestConfig.getStreamName(),
+        streamEventWriter,
+        20,
+        interval.getStart()
+    );
+    LOG.info("Generated %d Rows for Interval [%s]", rowCount, interval);
+
+    return rowCount;
+  }
+
+  /**
+   * Retries until segments have been loaded.
+   */
+  private void ensureSegmentsLoaded(String datasource)
+  {
+    ITRetryUtil.retryUntilTrue(
+        () -> coordinator.areSegmentsLoaded(datasource),
+        "Segment Loading"
+    );
+  }
+
+  private void checkAndSetConcurrentLocks(String datasource)
+  {
+    LockFilterPolicy lockFilterPolicy = new LockFilterPolicy(datasource, 0, 
null, null);
+    final List<TaskLock> locks = 
indexer.getActiveLocks(ImmutableList.of(lockFilterPolicy))
+                                        .getDatasourceToLocks()
+                                        .get(datasource);
+    if (CollectionUtils.isNullOrEmpty(locks)) {
+      return;
+    }
+    LOG.info(locks.toString());

Review Comment:
   Please add more info in this log line.



##########
integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITConcurrentAppendReplaceTest.java:
##########
@@ -0,0 +1,541 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.tests.coordinator.duty;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+import org.apache.druid.indexer.partitions.PartitionsSpec;
+import org.apache.druid.indexing.common.TaskLock;
+import org.apache.druid.indexing.common.TaskLockType;
+import org.apache.druid.indexing.common.task.Tasks;
+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.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.metadata.LockFilterPolicy;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
+import org.apache.druid.server.coordinator.DruidCompactionConfig;
+import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig;
+import org.apache.druid.testing.clients.CompactionResourceTestClient;
+import org.apache.druid.testing.clients.TaskResponseObject;
+import org.apache.druid.testing.guice.DruidTestModuleFactory;
+import org.apache.druid.testing.utils.EventSerializer;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.KafkaUtil;
+import org.apache.druid.testing.utils.MsqTestQueryHelper;
+import org.apache.druid.testing.utils.StreamEventWriter;
+import org.apache.druid.testing.utils.StreamGenerator;
+import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator;
+import org.apache.druid.tests.TestNGGroup;
+import org.apache.druid.tests.indexer.AbstractKafkaIndexingServiceTest;
+import org.apache.druid.tests.indexer.AbstractStreamIndexingTest;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.utils.CollectionUtils;
+import org.joda.time.Interval;
+import org.joda.time.Period;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+
+/**
+ * Integration Test to verify behaviour when there are concurrent
+ * compaction tasks with ongoing stream ingestion tasks.
+ */
+@Test(groups = {TestNGGroup.COMPACTION})
+@Guice(moduleFactory = DruidTestModuleFactory.class)
+public class ITConcurrentAppendReplaceTest extends 
AbstractKafkaIndexingServiceTest
+{
+  private static final Logger LOG = new 
Logger(ITConcurrentAppendReplaceTest.class);
+
+  @Inject
+  private CompactionResourceTestClient compactionResource;
+  
+  @Inject
+  private MsqTestQueryHelper msqHelper;
+
+  private GeneratedTestConfig generatedTestConfig;
+  private StreamGenerator streamGenerator;
+
+  private String fullDatasourceName;
+
+  private int currentRowCount = 0;
+  private boolean concurrentAppendAndReplaceLocksExisted;
+
+  @DataProvider
+  public static Object[] getParameters()
+  {
+    return new Object[]{false, true};
+  }
+
+  @BeforeClass
+  public void setupClass() throws Exception
+  {
+    doBeforeClass();
+  }
+
+  @BeforeMethod
+  public void setup() throws Exception
+  {
+    generatedTestConfig = new GeneratedTestConfig(
+        Specs.PARSER_TYPE,
+        getResourceAsString(Specs.INPUT_FORMAT_PATH)
+    );
+    fullDatasourceName = generatedTestConfig.getFullDatasourceName();
+    final EventSerializer serializer = jsonMapper.readValue(
+        getResourceAsStream(Specs.SERIALIZER_PATH),
+        EventSerializer.class
+    );
+    streamGenerator = new WikipediaStreamEventStreamGenerator(serializer, 6, 
100);
+    concurrentAppendAndReplaceLocksExisted = false;
+  }
+
+  @Override
+  public String getTestNamePrefix()
+  {
+    return "autocompact_lock_contention";
+  }
+
+  @Test(dataProvider = "getParameters")
+  public void testConcurrentStreamAppendReplace(boolean transactionEnabled) 
throws Exception
+  {
+    if (shouldSkipTest(transactionEnabled)) {
+      return;
+    }
+
+    try (
+        final Closeable closer = createResourceCloser(generatedTestConfig);
+        final StreamEventWriter streamEventWriter = 
createStreamEventWriter(config, transactionEnabled)
+    ) {
+      // Start supervisor
+      final String taskSpec
+          = generatedTestConfig.getStreamIngestionPropsTransform()
+                               
.apply(getResourceAsString(SUPERVISOR_WITH_CONCURRENT_LOCKS_SPEC_TEMPLATE_PATH));
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("supervisorSpec: [%s]", taskSpec);
+
+      // Generate data for minutes 1, 2 and 3
+      final Interval minute1 = 
Intervals.of("2000-01-01T01:01:00Z/2000-01-01T01:02:00Z");
+      long rowsForMinute1 = generateData(minute1, streamEventWriter);
+
+      final Interval minute2 = 
Intervals.of("2000-01-01T01:02:00Z/2000-01-01T01:03:00Z");
+      long rowsForMinute2 = generateData(minute2, streamEventWriter);
+
+      final Interval minute3 = 
Intervals.of("2000-01-01T01:03:00Z/2000-01-01T01:04:00Z");
+      long rowsForMinute3 = generateData(minute3, streamEventWriter);
+
+      Function<String, AggregatorFactory> function = name -> new 
LongSumAggregatorFactory(name, "count");
+
+      // Wait for data to be ingested for all the minutes
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+
+      ensureSegmentsLoaded(fullDatasourceName);
+
+      // 2 segments for each minute, total 6
+      ensureSegmentsCount(fullDatasourceName, 6);
+      printTaskStatuses(fullDatasourceName);
+
+      // Trigger auto compaction
+      submitAndVerifyCompactionConfig(fullDatasourceName, null);
+      compactionResource.forceTriggerAutoCompaction();
+
+      // Verify that all the segments are now compacted
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+      ensureSegmentsLoaded(fullDatasourceName);
+      ensureSegmentsCount(fullDatasourceName, 3);
+      verifyCompactedIntervals(fullDatasourceName, minute1, minute2, minute3);
+      printTaskStatuses(fullDatasourceName);
+
+      // Concurrent compaction with configured segment granularity
+      for (int i = 0; i < 5; i++) {
+        rowsForMinute1 += generateData(minute1, streamEventWriter);
+        rowsForMinute2 += generateData(minute2, streamEventWriter);
+        rowsForMinute3 += generateData(minute3, streamEventWriter);
+
+        //ensureNoLockContention();
+        compactionResource.forceTriggerAutoCompaction();
+        ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+        printTaskStatuses(fullDatasourceName);
+        checkAndSetConcurrentLocks(fullDatasourceName);
+      }
+
+      // Verify the state with minute granularity
+      ensureSegmentsCount(fullDatasourceName, 3);
+      ensureSegmentsLoaded(fullDatasourceName);
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+      verifyCompactedIntervals(fullDatasourceName, minute1, minute2, minute3);
+
+
+      // Use ALL segment granularity for compaction and run concurrent 
streaming ingestion
+      submitAndVerifyCompactionConfig(fullDatasourceName, Granularities.DAY);
+
+      for (int i = 0; i < 5; i++) {
+        rowsForMinute1 += generateData(minute1, streamEventWriter);
+        rowsForMinute2 += generateData(minute2, streamEventWriter);
+        rowsForMinute3 += generateData(minute3, streamEventWriter);
+
+        //ensureNoLockContention();
+        compactionResource.forceTriggerAutoCompaction();
+        ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+        printTaskStatuses(fullDatasourceName);
+        checkAndSetConcurrentLocks(fullDatasourceName);
+      }
+
+      // Verify the state with all granularity
+      ensureSegmentsCount(fullDatasourceName, 1);
+      ensureSegmentsLoaded(fullDatasourceName);
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+      verifyCompactedIntervals(fullDatasourceName, 
Intervals.of("2000-01-01/2000-01-02"));
+      printTaskStatuses(fullDatasourceName);
+
+      Assert.assertTrue(concurrentAppendAndReplaceLocksExisted);
+    }
+  }
+
+
+  @Test(dataProvider = "getParameters")
+  public void testConcurrentMSQAppendReplace(boolean transactionEnabled) 
throws Exception
+  {
+    if (shouldSkipTest(transactionEnabled)) {
+      return;
+    }

Review Comment:
   ```suggestion
       Assume.assumeFalse(shouldSkipTest(transactionEnabled));
   ```



##########
integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITConcurrentAppendReplaceTest.java:
##########
@@ -0,0 +1,541 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.tests.coordinator.duty;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+import org.apache.druid.indexer.partitions.PartitionsSpec;
+import org.apache.druid.indexing.common.TaskLock;
+import org.apache.druid.indexing.common.TaskLockType;
+import org.apache.druid.indexing.common.task.Tasks;
+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.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.metadata.LockFilterPolicy;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
+import org.apache.druid.server.coordinator.DruidCompactionConfig;
+import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig;
+import org.apache.druid.testing.clients.CompactionResourceTestClient;
+import org.apache.druid.testing.clients.TaskResponseObject;
+import org.apache.druid.testing.guice.DruidTestModuleFactory;
+import org.apache.druid.testing.utils.EventSerializer;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.KafkaUtil;
+import org.apache.druid.testing.utils.MsqTestQueryHelper;
+import org.apache.druid.testing.utils.StreamEventWriter;
+import org.apache.druid.testing.utils.StreamGenerator;
+import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator;
+import org.apache.druid.tests.TestNGGroup;
+import org.apache.druid.tests.indexer.AbstractKafkaIndexingServiceTest;
+import org.apache.druid.tests.indexer.AbstractStreamIndexingTest;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.utils.CollectionUtils;
+import org.joda.time.Interval;
+import org.joda.time.Period;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+
+/**
+ * Integration Test to verify behaviour when there are concurrent
+ * compaction tasks with ongoing stream ingestion tasks.
+ */
+@Test(groups = {TestNGGroup.COMPACTION})
+@Guice(moduleFactory = DruidTestModuleFactory.class)
+public class ITConcurrentAppendReplaceTest extends 
AbstractKafkaIndexingServiceTest
+{
+  private static final Logger LOG = new 
Logger(ITConcurrentAppendReplaceTest.class);
+
+  @Inject
+  private CompactionResourceTestClient compactionResource;
+  
+  @Inject
+  private MsqTestQueryHelper msqHelper;
+
+  private GeneratedTestConfig generatedTestConfig;
+  private StreamGenerator streamGenerator;
+
+  private String fullDatasourceName;
+
+  private int currentRowCount = 0;
+  private boolean concurrentAppendAndReplaceLocksExisted;
+
+  @DataProvider
+  public static Object[] getParameters()
+  {
+    return new Object[]{false, true};
+  }
+
+  @BeforeClass
+  public void setupClass() throws Exception
+  {
+    doBeforeClass();
+  }
+
+  @BeforeMethod
+  public void setup() throws Exception
+  {
+    generatedTestConfig = new GeneratedTestConfig(
+        Specs.PARSER_TYPE,
+        getResourceAsString(Specs.INPUT_FORMAT_PATH)
+    );
+    fullDatasourceName = generatedTestConfig.getFullDatasourceName();
+    final EventSerializer serializer = jsonMapper.readValue(
+        getResourceAsStream(Specs.SERIALIZER_PATH),
+        EventSerializer.class
+    );
+    streamGenerator = new WikipediaStreamEventStreamGenerator(serializer, 6, 
100);
+    concurrentAppendAndReplaceLocksExisted = false;
+  }
+
+  @Override
+  public String getTestNamePrefix()
+  {
+    return "autocompact_lock_contention";
+  }
+
+  @Test(dataProvider = "getParameters")
+  public void testConcurrentStreamAppendReplace(boolean transactionEnabled) 
throws Exception
+  {
+    if (shouldSkipTest(transactionEnabled)) {
+      return;
+    }
+
+    try (
+        final Closeable closer = createResourceCloser(generatedTestConfig);
+        final StreamEventWriter streamEventWriter = 
createStreamEventWriter(config, transactionEnabled)
+    ) {
+      // Start supervisor
+      final String taskSpec
+          = generatedTestConfig.getStreamIngestionPropsTransform()
+                               
.apply(getResourceAsString(SUPERVISOR_WITH_CONCURRENT_LOCKS_SPEC_TEMPLATE_PATH));
+      generatedTestConfig.setSupervisorId(indexer.submitSupervisor(taskSpec));
+      LOG.info("supervisorSpec: [%s]", taskSpec);
+
+      // Generate data for minutes 1, 2 and 3
+      final Interval minute1 = 
Intervals.of("2000-01-01T01:01:00Z/2000-01-01T01:02:00Z");
+      long rowsForMinute1 = generateData(minute1, streamEventWriter);
+
+      final Interval minute2 = 
Intervals.of("2000-01-01T01:02:00Z/2000-01-01T01:03:00Z");
+      long rowsForMinute2 = generateData(minute2, streamEventWriter);
+
+      final Interval minute3 = 
Intervals.of("2000-01-01T01:03:00Z/2000-01-01T01:04:00Z");
+      long rowsForMinute3 = generateData(minute3, streamEventWriter);
+
+      Function<String, AggregatorFactory> function = name -> new 
LongSumAggregatorFactory(name, "count");
+
+      // Wait for data to be ingested for all the minutes
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+
+      ensureSegmentsLoaded(fullDatasourceName);
+
+      // 2 segments for each minute, total 6
+      ensureSegmentsCount(fullDatasourceName, 6);
+      printTaskStatuses(fullDatasourceName);
+
+      // Trigger auto compaction
+      submitAndVerifyCompactionConfig(fullDatasourceName, null);
+      compactionResource.forceTriggerAutoCompaction();
+
+      // Verify that all the segments are now compacted
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+      ensureSegmentsLoaded(fullDatasourceName);
+      ensureSegmentsCount(fullDatasourceName, 3);
+      verifyCompactedIntervals(fullDatasourceName, minute1, minute2, minute3);
+      printTaskStatuses(fullDatasourceName);
+
+      // Concurrent compaction with configured segment granularity
+      for (int i = 0; i < 5; i++) {
+        rowsForMinute1 += generateData(minute1, streamEventWriter);
+        rowsForMinute2 += generateData(minute2, streamEventWriter);
+        rowsForMinute3 += generateData(minute3, streamEventWriter);
+
+        //ensureNoLockContention();
+        compactionResource.forceTriggerAutoCompaction();
+        ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+        printTaskStatuses(fullDatasourceName);
+        checkAndSetConcurrentLocks(fullDatasourceName);
+      }
+
+      // Verify the state with minute granularity
+      ensureSegmentsCount(fullDatasourceName, 3);
+      ensureSegmentsLoaded(fullDatasourceName);
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+      verifyCompactedIntervals(fullDatasourceName, minute1, minute2, minute3);
+
+
+      // Use ALL segment granularity for compaction and run concurrent 
streaming ingestion
+      submitAndVerifyCompactionConfig(fullDatasourceName, Granularities.DAY);
+
+      for (int i = 0; i < 5; i++) {
+        rowsForMinute1 += generateData(minute1, streamEventWriter);
+        rowsForMinute2 += generateData(minute2, streamEventWriter);
+        rowsForMinute3 += generateData(minute3, streamEventWriter);
+
+        //ensureNoLockContention();
+        compactionResource.forceTriggerAutoCompaction();
+        ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+        printTaskStatuses(fullDatasourceName);
+        checkAndSetConcurrentLocks(fullDatasourceName);
+      }
+
+      // Verify the state with all granularity
+      ensureSegmentsCount(fullDatasourceName, 1);
+      ensureSegmentsLoaded(fullDatasourceName);
+      ensureRowCount(fullDatasourceName, rowsForMinute1 + rowsForMinute2 + 
rowsForMinute3, function);
+      verifyCompactedIntervals(fullDatasourceName, 
Intervals.of("2000-01-01/2000-01-02"));
+      printTaskStatuses(fullDatasourceName);
+
+      Assert.assertTrue(concurrentAppendAndReplaceLocksExisted);
+    }
+  }
+
+
+  @Test(dataProvider = "getParameters")
+  public void testConcurrentMSQAppendReplace(boolean transactionEnabled) 
throws Exception
+  {
+    if (shouldSkipTest(transactionEnabled)) {
+      return;
+    }
+
+    final String datasource = "dst";
+
+    final String queryLocal =
+        StringUtils.format(
+            "INSERT INTO %s\n"
+            + "SELECT\n"
+            + "  TIME_PARSE(\"timestamp\") AS __time,\n"
+            + "  isRobot,\n"
+            + "  diffUrl,\n"
+            + "  added,\n"
+            + "  countryIsoCode,\n"
+            + "  regionName,\n"
+            + "  channel,\n"
+            + "  flags,\n"
+            + "  delta,\n"
+            + "  isUnpatrolled,\n"
+            + "  isNew,\n"
+            + "  deltaBucket,\n"
+            + "  isMinor,\n"
+            + "  isAnonymous,\n"
+            + "  deleted,\n"
+            + "  cityName,\n"
+            + "  metroCode,\n"
+            + "  namespace,\n"
+            + "  comment,\n"
+            + "  page,\n"
+            + "  commentLength,\n"
+            + "  countryName,\n"
+            + "  user,\n"
+            + "  regionIsoCode\n"
+            + "FROM TABLE(\n"
+            + "  EXTERN(\n"
+            + "    
'{\"type\":\"local\",\"files\":[\"/resources/data/batch_index/json/wikipedia_index_data1.json\"]}',\n"
+            + "    '{\"type\":\"json\"}',\n"
+            + "    
'[{\"type\":\"string\",\"name\":\"timestamp\"},{\"type\":\"string\",\"name\":\"isRobot\"},{\"type\":\"string\",\"name\":\"diffUrl\"},{\"type\":\"long\",\"name\":\"added\"},{\"type\":\"string\",\"name\":\"countryIsoCode\"},{\"type\":\"string\",\"name\":\"regionName\"},{\"type\":\"string\",\"name\":\"channel\"},{\"type\":\"string\",\"name\":\"flags\"},{\"type\":\"long\",\"name\":\"delta\"},{\"type\":\"string\",\"name\":\"isUnpatrolled\"},{\"type\":\"string\",\"name\":\"isNew\"},{\"type\":\"double\",\"name\":\"deltaBucket\"},{\"type\":\"string\",\"name\":\"isMinor\"},{\"type\":\"string\",\"name\":\"isAnonymous\"},{\"type\":\"long\",\"name\":\"deleted\"},{\"type\":\"string\",\"name\":\"cityName\"},{\"type\":\"long\",\"name\":\"metroCode\"},{\"type\":\"string\",\"name\":\"namespace\"},{\"type\":\"string\",\"name\":\"comment\"},{\"type\":\"string\",\"name\":\"page\"},{\"type\":\"long\",\"name\":\"commentLength\"},{\"type\":\"string\",\"name\":\"countryName\"},{\"type\":
 
\"string\",\"name\":\"user\"},{\"type\":\"string\",\"name\":\"regionIsoCode\"}]'\n"
+            + "  )\n"
+            + ")\n"
+            + "PARTITIONED BY DAY\n",
+            datasource
+        );
+
+    Function<String, AggregatorFactory> function = name -> new 
CountAggregatorFactory("rows");
+
+    msqHelper.submitMsqTaskSuccesfully(queryLocal, 
ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true));
+    ensureSegmentsCount(datasource, 1);
+    ensureRowCount(datasource, 3, function);
+
+    submitAndVerifyCompactionConfig(datasource, null);
+
+    for (int i = 0; i < 5; i++) {
+      // Submit the task and wait for the datasource to get loaded
+      msqHelper.submitMsqTaskSuccesfully(queryLocal, 
ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true));
+
+      ensureRowCount(datasource, (i + 2) * 3, function);
+    }

Review Comment:
   Don't we need to `forceTriggerCompaction` for this test?



##########
integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITConcurrentAppendReplaceTest.java:
##########
@@ -0,0 +1,541 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.tests.coordinator.duty;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+import org.apache.druid.indexer.partitions.PartitionsSpec;
+import org.apache.druid.indexing.common.TaskLock;
+import org.apache.druid.indexing.common.TaskLockType;
+import org.apache.druid.indexing.common.task.Tasks;
+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.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.metadata.LockFilterPolicy;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
+import org.apache.druid.server.coordinator.DruidCompactionConfig;
+import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig;
+import org.apache.druid.testing.clients.CompactionResourceTestClient;
+import org.apache.druid.testing.clients.TaskResponseObject;
+import org.apache.druid.testing.guice.DruidTestModuleFactory;
+import org.apache.druid.testing.utils.EventSerializer;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.KafkaUtil;
+import org.apache.druid.testing.utils.MsqTestQueryHelper;
+import org.apache.druid.testing.utils.StreamEventWriter;
+import org.apache.druid.testing.utils.StreamGenerator;
+import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator;
+import org.apache.druid.tests.TestNGGroup;
+import org.apache.druid.tests.indexer.AbstractKafkaIndexingServiceTest;
+import org.apache.druid.tests.indexer.AbstractStreamIndexingTest;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.utils.CollectionUtils;
+import org.joda.time.Interval;
+import org.joda.time.Period;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+
+/**
+ * Integration Test to verify behaviour when there are concurrent
+ * compaction tasks with ongoing stream ingestion tasks.
+ */
+@Test(groups = {TestNGGroup.COMPACTION})
+@Guice(moduleFactory = DruidTestModuleFactory.class)
+public class ITConcurrentAppendReplaceTest extends 
AbstractKafkaIndexingServiceTest
+{
+  private static final Logger LOG = new 
Logger(ITConcurrentAppendReplaceTest.class);
+
+  @Inject
+  private CompactionResourceTestClient compactionResource;
+  
+  @Inject
+  private MsqTestQueryHelper msqHelper;
+
+  private GeneratedTestConfig generatedTestConfig;
+  private StreamGenerator streamGenerator;
+
+  private String fullDatasourceName;
+
+  private int currentRowCount = 0;
+  private boolean concurrentAppendAndReplaceLocksExisted;
+
+  @DataProvider
+  public static Object[] getParameters()

Review Comment:
   Maybe rename this to something more meaningful like 
`getKafkaTransactionStatesForTest()`



##########
integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITConcurrentAppendReplaceTest.java:
##########
@@ -0,0 +1,541 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.tests.coordinator.duty;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+import org.apache.druid.indexer.partitions.PartitionsSpec;
+import org.apache.druid.indexing.common.TaskLock;
+import org.apache.druid.indexing.common.TaskLockType;
+import org.apache.druid.indexing.common.task.Tasks;
+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.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.metadata.LockFilterPolicy;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
+import org.apache.druid.server.coordinator.DruidCompactionConfig;
+import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig;
+import org.apache.druid.testing.clients.CompactionResourceTestClient;
+import org.apache.druid.testing.clients.TaskResponseObject;
+import org.apache.druid.testing.guice.DruidTestModuleFactory;
+import org.apache.druid.testing.utils.EventSerializer;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.KafkaUtil;
+import org.apache.druid.testing.utils.MsqTestQueryHelper;
+import org.apache.druid.testing.utils.StreamEventWriter;
+import org.apache.druid.testing.utils.StreamGenerator;
+import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator;
+import org.apache.druid.tests.TestNGGroup;
+import org.apache.druid.tests.indexer.AbstractKafkaIndexingServiceTest;
+import org.apache.druid.tests.indexer.AbstractStreamIndexingTest;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.utils.CollectionUtils;
+import org.joda.time.Interval;
+import org.joda.time.Period;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+
+/**
+ * Integration Test to verify behaviour when there are concurrent
+ * compaction tasks with ongoing stream ingestion tasks.
+ */
+@Test(groups = {TestNGGroup.COMPACTION})
+@Guice(moduleFactory = DruidTestModuleFactory.class)
+public class ITConcurrentAppendReplaceTest extends 
AbstractKafkaIndexingServiceTest
+{
+  private static final Logger LOG = new 
Logger(ITConcurrentAppendReplaceTest.class);
+
+  @Inject
+  private CompactionResourceTestClient compactionResource;
+  
+  @Inject
+  private MsqTestQueryHelper msqHelper;
+
+  private GeneratedTestConfig generatedTestConfig;
+  private StreamGenerator streamGenerator;
+
+  private String fullDatasourceName;
+
+  private int currentRowCount = 0;
+  private boolean concurrentAppendAndReplaceLocksExisted;

Review Comment:
   Why is this a field of the class? The `checkAndSetConcurrentLocks` method 
should return a boolean instead and the value should be verified in the 
relevant test.



##########
integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITConcurrentAppendReplaceTest.java:
##########
@@ -0,0 +1,541 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.tests.coordinator.duty;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.Inject;
+import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
+import org.apache.druid.indexer.partitions.PartitionsSpec;
+import org.apache.druid.indexing.common.TaskLock;
+import org.apache.druid.indexing.common.TaskLockType;
+import org.apache.druid.indexing.common.task.Tasks;
+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.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.metadata.LockFilterPolicy;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
+import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
+import org.apache.druid.server.coordinator.DruidCompactionConfig;
+import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig;
+import org.apache.druid.testing.clients.CompactionResourceTestClient;
+import org.apache.druid.testing.clients.TaskResponseObject;
+import org.apache.druid.testing.guice.DruidTestModuleFactory;
+import org.apache.druid.testing.utils.EventSerializer;
+import org.apache.druid.testing.utils.ITRetryUtil;
+import org.apache.druid.testing.utils.KafkaUtil;
+import org.apache.druid.testing.utils.MsqTestQueryHelper;
+import org.apache.druid.testing.utils.StreamEventWriter;
+import org.apache.druid.testing.utils.StreamGenerator;
+import org.apache.druid.testing.utils.WikipediaStreamEventStreamGenerator;
+import org.apache.druid.tests.TestNGGroup;
+import org.apache.druid.tests.indexer.AbstractKafkaIndexingServiceTest;
+import org.apache.druid.tests.indexer.AbstractStreamIndexingTest;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.utils.CollectionUtils;
+import org.joda.time.Interval;
+import org.joda.time.Period;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Guice;
+import org.testng.annotations.Test;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+
+/**
+ * Integration Test to verify behaviour when there are concurrent
+ * compaction tasks with ongoing stream ingestion tasks.
+ */
+@Test(groups = {TestNGGroup.COMPACTION})
+@Guice(moduleFactory = DruidTestModuleFactory.class)
+public class ITConcurrentAppendReplaceTest extends 
AbstractKafkaIndexingServiceTest
+{
+  private static final Logger LOG = new 
Logger(ITConcurrentAppendReplaceTest.class);
+
+  @Inject
+  private CompactionResourceTestClient compactionResource;
+  
+  @Inject
+  private MsqTestQueryHelper msqHelper;
+
+  private GeneratedTestConfig generatedTestConfig;
+  private StreamGenerator streamGenerator;
+
+  private String fullDatasourceName;
+
+  private int currentRowCount = 0;
+  private boolean concurrentAppendAndReplaceLocksExisted;
+
+  @DataProvider
+  public static Object[] getParameters()
+  {
+    return new Object[]{false, true};
+  }
+
+  @BeforeClass
+  public void setupClass() throws Exception
+  {
+    doBeforeClass();
+  }
+
+  @BeforeMethod
+  public void setup() throws Exception
+  {
+    generatedTestConfig = new GeneratedTestConfig(
+        Specs.PARSER_TYPE,
+        getResourceAsString(Specs.INPUT_FORMAT_PATH)
+    );
+    fullDatasourceName = generatedTestConfig.getFullDatasourceName();
+    final EventSerializer serializer = jsonMapper.readValue(
+        getResourceAsStream(Specs.SERIALIZER_PATH),
+        EventSerializer.class
+    );
+    streamGenerator = new WikipediaStreamEventStreamGenerator(serializer, 6, 
100);
+    concurrentAppendAndReplaceLocksExisted = false;
+  }
+
+  @Override
+  public String getTestNamePrefix()
+  {
+    return "autocompact_lock_contention";
+  }
+
+  @Test(dataProvider = "getParameters")
+  public void testConcurrentStreamAppendReplace(boolean transactionEnabled) 
throws Exception
+  {
+    if (shouldSkipTest(transactionEnabled)) {
+      return;
+    }

Review Comment:
   ```suggestion
       Assume.assumeFalse(shouldSkipTest(transactionEnabled));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@druid.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@druid.apache.org
For additional commands, e-mail: commits-h...@druid.apache.org

Reply via email to