mirror of https://github.com/apache/druid.git
Fix exclusive start partitions for sequenceMetadata (#7339)
* Fix exclusvie start partitions for sequenceMetadata * add empty check
This commit is contained in:
parent
105b4fa237
commit
5294277cb4
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Joiner;
|
import com.google.common.base.Joiner;
|
||||||
import com.google.common.collect.ImmutableMap;
|
|
||||||
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
|
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
|
||||||
import org.apache.druid.indexing.common.task.Task;
|
import org.apache.druid.indexing.common.task.Task;
|
||||||
import org.apache.druid.indexing.common.task.TaskResource;
|
import org.apache.druid.indexing.common.task.TaskResource;
|
||||||
|
@ -80,6 +79,11 @@ import java.util.stream.Collectors;
|
||||||
*/
|
*/
|
||||||
public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long>
|
public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long>
|
||||||
{
|
{
|
||||||
|
public static final TypeReference<TreeMap<Integer, Map<Integer, Long>>> CHECKPOINTS_TYPE_REF =
|
||||||
|
new TypeReference<TreeMap<Integer, Map<Integer, Long>>>()
|
||||||
|
{
|
||||||
|
};
|
||||||
|
|
||||||
private static final EmittingLogger log = new EmittingLogger(KafkaSupervisor.class);
|
private static final EmittingLogger log = new EmittingLogger(KafkaSupervisor.class);
|
||||||
private static final long MINIMUM_GET_OFFSET_PERIOD_MILLIS = 5000;
|
private static final long MINIMUM_GET_OFFSET_PERIOD_MILLIS = 5000;
|
||||||
private static final long INITIAL_GET_OFFSET_DELAY_MILLIS = 15000;
|
private static final long INITIAL_GET_OFFSET_DELAY_MILLIS = 15000;
|
||||||
|
@ -229,20 +233,9 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long>
|
||||||
RowIngestionMetersFactory rowIngestionMetersFactory
|
RowIngestionMetersFactory rowIngestionMetersFactory
|
||||||
) throws JsonProcessingException
|
) throws JsonProcessingException
|
||||||
{
|
{
|
||||||
final String checkpoints = sortingMapper.writerWithType(new TypeReference<TreeMap<Integer, Map<Integer, Long>>>()
|
final String checkpoints = sortingMapper.writerFor(CHECKPOINTS_TYPE_REF).writeValueAsString(sequenceOffsets);
|
||||||
{
|
final Map<String, Object> context = createBaseTaskContexts();
|
||||||
}).writeValueAsString(sequenceOffsets);
|
context.put(CHECKPOINTS_CTX_KEY, checkpoints);
|
||||||
final Map<String, Object> context = spec.getContext() == null
|
|
||||||
? ImmutableMap.of(
|
|
||||||
"checkpoints",
|
|
||||||
checkpoints,
|
|
||||||
IS_INCREMENTAL_HANDOFF_SUPPORTED,
|
|
||||||
true
|
|
||||||
) : ImmutableMap.<String, Object>builder()
|
|
||||||
.put("checkpoints", checkpoints)
|
|
||||||
.put(IS_INCREMENTAL_HANDOFF_SUPPORTED, true)
|
|
||||||
.putAll(spec.getContext())
|
|
||||||
.build();
|
|
||||||
|
|
||||||
List<SeekableStreamIndexTask<Integer, Long>> taskList = new ArrayList<>();
|
List<SeekableStreamIndexTask<Integer, Long>> taskList = new ArrayList<>();
|
||||||
for (int i = 0; i < replicas; i++) {
|
for (int i = 0; i < replicas; i++) {
|
||||||
|
@ -358,7 +351,7 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected boolean useExclusiveStartSequenceNumberForStartSequence()
|
protected boolean useExclusiveStartSequenceNumberForNonFirstSequence()
|
||||||
{
|
{
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package org.apache.druid.indexing.kafka;
|
package org.apache.druid.indexing.kafka;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||||
import com.fasterxml.jackson.core.type.TypeReference;
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import com.fasterxml.jackson.databind.Module;
|
import com.fasterxml.jackson.databind.Module;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
@ -67,6 +68,7 @@ import org.apache.druid.indexing.common.stats.RowIngestionMeters;
|
||||||
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
|
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskTest;
|
import org.apache.druid.indexing.common.task.IndexTaskTest;
|
||||||
import org.apache.druid.indexing.common.task.Task;
|
import org.apache.druid.indexing.common.task.Task;
|
||||||
|
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisor;
|
||||||
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig;
|
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig;
|
||||||
import org.apache.druid.indexing.kafka.test.TestBroker;
|
import org.apache.druid.indexing.kafka.test.TestBroker;
|
||||||
import org.apache.druid.indexing.overlord.DataSourceMetadata;
|
import org.apache.druid.indexing.overlord.DataSourceMetadata;
|
||||||
|
@ -2143,9 +2145,10 @@ public class KafkaIndexTaskTest
|
||||||
// and this task should start reading from offset 2 for partition 0
|
// and this task should start reading from offset 2 for partition 0
|
||||||
sequences.put(1, ImmutableMap.of(0, 2L));
|
sequences.put(1, ImmutableMap.of(0, 2L));
|
||||||
final Map<String, Object> context = new HashMap<>();
|
final Map<String, Object> context = new HashMap<>();
|
||||||
context.put("checkpoints", objectMapper.writerWithType(new TypeReference<TreeMap<Integer, Map<Integer, Long>>>()
|
context.put(
|
||||||
{
|
SeekableStreamSupervisor.CHECKPOINTS_CTX_KEY,
|
||||||
}).writeValueAsString(sequences));
|
objectMapper.writerFor(KafkaSupervisor.CHECKPOINTS_TYPE_REF).writeValueAsString(sequences)
|
||||||
|
);
|
||||||
|
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
|
@ -2454,7 +2457,7 @@ public class KafkaIndexTaskTest
|
||||||
private KafkaIndexTask createTask(
|
private KafkaIndexTask createTask(
|
||||||
final String taskId,
|
final String taskId,
|
||||||
final KafkaIndexTaskIOConfig ioConfig
|
final KafkaIndexTaskIOConfig ioConfig
|
||||||
)
|
) throws JsonProcessingException
|
||||||
{
|
{
|
||||||
return createTask(taskId, DATA_SCHEMA, ioConfig);
|
return createTask(taskId, DATA_SCHEMA, ioConfig);
|
||||||
}
|
}
|
||||||
|
@ -2463,7 +2466,7 @@ public class KafkaIndexTaskTest
|
||||||
final String taskId,
|
final String taskId,
|
||||||
final KafkaIndexTaskIOConfig ioConfig,
|
final KafkaIndexTaskIOConfig ioConfig,
|
||||||
final Map<String, Object> context
|
final Map<String, Object> context
|
||||||
)
|
) throws JsonProcessingException
|
||||||
{
|
{
|
||||||
return createTask(taskId, DATA_SCHEMA, ioConfig, context);
|
return createTask(taskId, DATA_SCHEMA, ioConfig, context);
|
||||||
}
|
}
|
||||||
|
@ -2472,7 +2475,18 @@ public class KafkaIndexTaskTest
|
||||||
final String taskId,
|
final String taskId,
|
||||||
final DataSchema dataSchema,
|
final DataSchema dataSchema,
|
||||||
final KafkaIndexTaskIOConfig ioConfig
|
final KafkaIndexTaskIOConfig ioConfig
|
||||||
)
|
) throws JsonProcessingException
|
||||||
|
{
|
||||||
|
final Map<String, Object> context = new HashMap<>();
|
||||||
|
return createTask(taskId, dataSchema, ioConfig, context);
|
||||||
|
}
|
||||||
|
|
||||||
|
private KafkaIndexTask createTask(
|
||||||
|
final String taskId,
|
||||||
|
final DataSchema dataSchema,
|
||||||
|
final KafkaIndexTaskIOConfig ioConfig,
|
||||||
|
final Map<String, Object> context
|
||||||
|
) throws JsonProcessingException
|
||||||
{
|
{
|
||||||
final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig(
|
final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig(
|
||||||
1000,
|
1000,
|
||||||
|
@ -2493,57 +2507,17 @@ public class KafkaIndexTaskTest
|
||||||
maxParseExceptions,
|
maxParseExceptions,
|
||||||
maxSavedParseExceptions
|
maxSavedParseExceptions
|
||||||
);
|
);
|
||||||
final Map<String, Object> context = isIncrementalHandoffSupported
|
|
||||||
? ImmutableMap.of(
|
|
||||||
SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED,
|
|
||||||
true
|
|
||||||
)
|
|
||||||
: null;
|
|
||||||
final KafkaIndexTask task = new KafkaIndexTask(
|
|
||||||
taskId,
|
|
||||||
null,
|
|
||||||
cloneDataSchema(dataSchema),
|
|
||||||
tuningConfig,
|
|
||||||
ioConfig,
|
|
||||||
context,
|
|
||||||
null,
|
|
||||||
null,
|
|
||||||
rowIngestionMetersFactory,
|
|
||||||
objectMapper
|
|
||||||
);
|
|
||||||
task.setPollRetryMs(POLL_RETRY_MS);
|
|
||||||
return task;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
private KafkaIndexTask createTask(
|
|
||||||
final String taskId,
|
|
||||||
final DataSchema dataSchema,
|
|
||||||
final KafkaIndexTaskIOConfig ioConfig,
|
|
||||||
final Map<String, Object> context
|
|
||||||
)
|
|
||||||
{
|
|
||||||
final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig(
|
|
||||||
1000,
|
|
||||||
null,
|
|
||||||
maxRowsPerSegment,
|
|
||||||
null,
|
|
||||||
new Period("P1Y"),
|
|
||||||
null,
|
|
||||||
null,
|
|
||||||
null,
|
|
||||||
true,
|
|
||||||
reportParseExceptions,
|
|
||||||
handoffConditionTimeout,
|
|
||||||
resetOffsetAutomatically,
|
|
||||||
null,
|
|
||||||
null,
|
|
||||||
logParseExceptions,
|
|
||||||
maxParseExceptions,
|
|
||||||
maxSavedParseExceptions
|
|
||||||
);
|
|
||||||
if (isIncrementalHandoffSupported) {
|
if (isIncrementalHandoffSupported) {
|
||||||
context.put(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true);
|
context.put(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true);
|
||||||
|
|
||||||
|
if (!context.containsKey(SeekableStreamSupervisor.CHECKPOINTS_CTX_KEY)) {
|
||||||
|
final TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
||||||
|
checkpoints.put(0, ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap());
|
||||||
|
final String checkpointsJson = objectMapper
|
||||||
|
.writerFor(KafkaSupervisor.CHECKPOINTS_TYPE_REF)
|
||||||
|
.writeValueAsString(checkpoints);
|
||||||
|
context.put(SeekableStreamSupervisor.CHECKPOINTS_CTX_KEY, checkpointsJson);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
final KafkaIndexTask task = new KafkaIndexTask(
|
final KafkaIndexTask task = new KafkaIndexTask(
|
||||||
|
@ -2629,8 +2603,8 @@ public class KafkaIndexTaskTest
|
||||||
objectMapper.registerModule(module);
|
objectMapper.registerModule(module);
|
||||||
}
|
}
|
||||||
final TaskConfig taskConfig = new TaskConfig(
|
final TaskConfig taskConfig = new TaskConfig(
|
||||||
new File(directory, "taskBaseDir").getPath(),
|
new File(directory, "baseDir").getPath(),
|
||||||
null,
|
new File(directory, "baseTaskDir").getPath(),
|
||||||
null,
|
null,
|
||||||
50000,
|
50000,
|
||||||
null,
|
null,
|
||||||
|
|
|
@ -75,6 +75,11 @@ import java.util.concurrent.ScheduledExecutorService;
|
||||||
*/
|
*/
|
||||||
public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
|
public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
|
||||||
{
|
{
|
||||||
|
public static final TypeReference<TreeMap<Integer, Map<String, String>>> CHECKPOINTS_TYPE_REF =
|
||||||
|
new TypeReference<TreeMap<Integer, Map<String, String>>>()
|
||||||
|
{
|
||||||
|
};
|
||||||
|
|
||||||
private static final String NOT_SET = "-1";
|
private static final String NOT_SET = "-1";
|
||||||
private final KinesisSupervisorSpec spec;
|
private final KinesisSupervisorSpec spec;
|
||||||
private final AWSCredentialsConfig awsCredentialsConfig;
|
private final AWSCredentialsConfig awsCredentialsConfig;
|
||||||
|
@ -151,20 +156,10 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
|
||||||
RowIngestionMetersFactory rowIngestionMetersFactory
|
RowIngestionMetersFactory rowIngestionMetersFactory
|
||||||
) throws JsonProcessingException
|
) throws JsonProcessingException
|
||||||
{
|
{
|
||||||
final String checkpoints = sortingMapper.writerFor(new TypeReference<TreeMap<Integer, Map<String, String>>>()
|
final String checkpoints = sortingMapper.writerFor(CHECKPOINTS_TYPE_REF).writeValueAsString(sequenceOffsets);
|
||||||
{
|
final Map<String, Object> context = createBaseTaskContexts();
|
||||||
}).writeValueAsString(sequenceOffsets);
|
context.put(CHECKPOINTS_CTX_KEY, checkpoints);
|
||||||
final Map<String, Object> context = spec.getContext() == null
|
|
||||||
? ImmutableMap.of(
|
|
||||||
"checkpoints",
|
|
||||||
checkpoints,
|
|
||||||
IS_INCREMENTAL_HANDOFF_SUPPORTED,
|
|
||||||
true
|
|
||||||
) : ImmutableMap.<String, Object>builder()
|
|
||||||
.put("checkpoints", checkpoints)
|
|
||||||
.put(IS_INCREMENTAL_HANDOFF_SUPPORTED, true)
|
|
||||||
.putAll(spec.getContext())
|
|
||||||
.build();
|
|
||||||
List<SeekableStreamIndexTask<String, String>> taskList = new ArrayList<>();
|
List<SeekableStreamIndexTask<String, String>> taskList = new ArrayList<>();
|
||||||
for (int i = 0; i < replicas; i++) {
|
for (int i = 0; i < replicas; i++) {
|
||||||
String taskId = Joiner.on("_").join(baseSequenceName, RandomIdUtils.getRandomId());
|
String taskId = Joiner.on("_").join(baseSequenceName, RandomIdUtils.getRandomId());
|
||||||
|
@ -313,7 +308,7 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected boolean useExclusiveStartSequenceNumberForStartSequence()
|
protected boolean useExclusiveStartSequenceNumberForNonFirstSequence()
|
||||||
{
|
{
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||||
|
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||||
import com.fasterxml.jackson.core.type.TypeReference;
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import com.fasterxml.jackson.databind.InjectableValues;
|
import com.fasterxml.jackson.databind.InjectableValues;
|
||||||
import com.fasterxml.jackson.databind.Module;
|
import com.fasterxml.jackson.databind.Module;
|
||||||
|
@ -74,6 +75,7 @@ import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskTest;
|
import org.apache.druid.indexing.common.task.IndexTaskTest;
|
||||||
import org.apache.druid.indexing.common.task.Task;
|
import org.apache.druid.indexing.common.task.Task;
|
||||||
import org.apache.druid.indexing.common.task.TaskResource;
|
import org.apache.druid.indexing.common.task.TaskResource;
|
||||||
|
import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisor;
|
||||||
import org.apache.druid.indexing.overlord.DataSourceMetadata;
|
import org.apache.druid.indexing.overlord.DataSourceMetadata;
|
||||||
import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
|
import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
|
||||||
import org.apache.druid.indexing.overlord.MetadataTaskStorage;
|
import org.apache.druid.indexing.overlord.MetadataTaskStorage;
|
||||||
|
@ -83,6 +85,7 @@ import org.apache.druid.indexing.overlord.supervisor.SupervisorManager;
|
||||||
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
|
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
|
||||||
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
|
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner;
|
||||||
import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
|
import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers;
|
||||||
|
import org.apache.druid.indexing.seekablestream.SequenceMetadata;
|
||||||
import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
|
import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
|
||||||
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
|
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
|
||||||
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor;
|
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor;
|
||||||
|
@ -178,6 +181,7 @@ import java.util.Objects;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
|
import java.util.concurrent.CopyOnWriteArrayList;
|
||||||
import java.util.concurrent.Executor;
|
import java.util.concurrent.Executor;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
@ -509,7 +513,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||||
Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc2));
|
Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc2));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test(timeout = 120_000L)
|
@Test(timeout = 120_000L)
|
||||||
public void testIncrementalHandOff() throws Exception
|
public void testIncrementalHandOff() throws Exception
|
||||||
{
|
{
|
||||||
|
@ -736,7 +739,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||||
DATA_SCHEMA.getDataSource(),
|
DATA_SCHEMA.getDataSource(),
|
||||||
0,
|
0,
|
||||||
new KinesisDataSourceMetadata(
|
new KinesisDataSourceMetadata(
|
||||||
new SeekableStreamStartSequenceNumbers<>(stream, currentOffsets, ImmutableSet.of())
|
new SeekableStreamStartSequenceNumbers<>(stream, currentOffsets, currentOffsets.keySet())
|
||||||
),
|
),
|
||||||
new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(stream, nextOffsets))
|
new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(stream, nextOffsets))
|
||||||
)
|
)
|
||||||
|
@ -1949,7 +1952,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||||
new KinesisIndexTaskIOConfig(
|
new KinesisIndexTaskIOConfig(
|
||||||
null,
|
null,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of(shardId1)),
|
new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "2"), ImmutableSet.of()),
|
||||||
new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "5")),
|
new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "5")),
|
||||||
true,
|
true,
|
||||||
null,
|
null,
|
||||||
|
@ -2047,10 +2050,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
final SeekableStreamStartSequenceNumbers<String, String> checkpoint1 = new SeekableStreamStartSequenceNumbers<>(
|
final SeekableStreamEndSequenceNumbers<String, String> checkpoint1 = new SeekableStreamEndSequenceNumbers<>(
|
||||||
stream,
|
stream,
|
||||||
ImmutableMap.of(shardId1, "4"),
|
ImmutableMap.of(shardId1, "4")
|
||||||
ImmutableSet.of()
|
|
||||||
);
|
);
|
||||||
|
|
||||||
final ListenableFuture<TaskStatus> future1 = runTask(task1);
|
final ListenableFuture<TaskStatus> future1 = runTask(task1);
|
||||||
|
@ -2095,7 +2097,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||||
new KinesisIndexTaskIOConfig(
|
new KinesisIndexTaskIOConfig(
|
||||||
null,
|
null,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "0"), ImmutableSet.of(shardId1)),
|
new SeekableStreamStartSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "0"), ImmutableSet.of()),
|
||||||
new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "6")),
|
new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId1, "6")),
|
||||||
true,
|
true,
|
||||||
null,
|
null,
|
||||||
|
@ -2271,9 +2273,10 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||||
// and this task should start reading from offset 2 for partition 0 (not offset 1, because end is inclusive)
|
// and this task should start reading from offset 2 for partition 0 (not offset 1, because end is inclusive)
|
||||||
sequences.put(1, ImmutableMap.of(shardId1, "1"));
|
sequences.put(1, ImmutableMap.of(shardId1, "1"));
|
||||||
final Map<String, Object> context = new HashMap<>();
|
final Map<String, Object> context = new HashMap<>();
|
||||||
context.put("checkpoints", objectMapper.writerWithType(new TypeReference<TreeMap<Integer, Map<String, String>>>()
|
context.put(
|
||||||
{
|
SeekableStreamSupervisor.CHECKPOINTS_CTX_KEY,
|
||||||
}).writeValueAsString(sequences));
|
objectMapper.writerFor(KinesisSupervisor.CHECKPOINTS_TYPE_REF).writeValueAsString(sequences)
|
||||||
|
);
|
||||||
|
|
||||||
|
|
||||||
final KinesisIndexTask task = createTask(
|
final KinesisIndexTask task = createTask(
|
||||||
|
@ -2473,6 +2476,75 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSequencesFromContext() throws IOException
|
||||||
|
{
|
||||||
|
final TreeMap<Integer, Map<String, String>> checkpoints = new TreeMap<>();
|
||||||
|
// Here the sequence number is 1 meaning that one incremental handoff was done by the failed task
|
||||||
|
// and this task should start reading from offset 2 for partition 0 (not offset 1, because end is inclusive)
|
||||||
|
checkpoints.put(0, ImmutableMap.of(shardId0, "0", shardId1, "0"));
|
||||||
|
checkpoints.put(1, ImmutableMap.of(shardId0, "0", shardId1, "1"));
|
||||||
|
checkpoints.put(2, ImmutableMap.of(shardId0, "1", shardId1, "3"));
|
||||||
|
final Map<String, Object> context = new HashMap<>();
|
||||||
|
context.put("checkpoints", objectMapper.writerFor(KinesisSupervisor.CHECKPOINTS_TYPE_REF)
|
||||||
|
.writeValueAsString(checkpoints));
|
||||||
|
|
||||||
|
final KinesisIndexTask task = createTask(
|
||||||
|
"task1",
|
||||||
|
DATA_SCHEMA,
|
||||||
|
new KinesisIndexTaskIOConfig(
|
||||||
|
null,
|
||||||
|
"sequence0",
|
||||||
|
new SeekableStreamStartSequenceNumbers<>(
|
||||||
|
stream,
|
||||||
|
ImmutableMap.of(shardId0, "0", shardId1, "0"),
|
||||||
|
ImmutableSet.of(shardId0)
|
||||||
|
),
|
||||||
|
new SeekableStreamEndSequenceNumbers<>(stream, ImmutableMap.of(shardId0, "1", shardId1, "5")),
|
||||||
|
true,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
"awsEndpoint",
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
false
|
||||||
|
),
|
||||||
|
context
|
||||||
|
);
|
||||||
|
|
||||||
|
task.getRunner().setToolbox(toolboxFactory.build(task));
|
||||||
|
task.getRunner().initializeSequences();
|
||||||
|
final CopyOnWriteArrayList<SequenceMetadata<String, String>> sequences = task.getRunner().getSequences();
|
||||||
|
|
||||||
|
Assert.assertEquals(3, sequences.size());
|
||||||
|
|
||||||
|
SequenceMetadata<String, String> sequenceMetadata = sequences.get(0);
|
||||||
|
Assert.assertEquals(checkpoints.get(0), sequenceMetadata.getStartOffsets());
|
||||||
|
Assert.assertEquals(checkpoints.get(1), sequenceMetadata.getEndOffsets());
|
||||||
|
Assert.assertEquals(
|
||||||
|
task.getIOConfig().getStartSequenceNumbers().getExclusivePartitions(),
|
||||||
|
sequenceMetadata.getExclusiveStartPartitions()
|
||||||
|
);
|
||||||
|
Assert.assertTrue(sequenceMetadata.isCheckpointed());
|
||||||
|
|
||||||
|
sequenceMetadata = sequences.get(1);
|
||||||
|
Assert.assertEquals(checkpoints.get(1), sequenceMetadata.getStartOffsets());
|
||||||
|
Assert.assertEquals(checkpoints.get(2), sequenceMetadata.getEndOffsets());
|
||||||
|
Assert.assertEquals(checkpoints.get(1).keySet(), sequenceMetadata.getExclusiveStartPartitions());
|
||||||
|
Assert.assertTrue(sequenceMetadata.isCheckpointed());
|
||||||
|
|
||||||
|
sequenceMetadata = sequences.get(2);
|
||||||
|
Assert.assertEquals(checkpoints.get(2), sequenceMetadata.getStartOffsets());
|
||||||
|
Assert.assertEquals(
|
||||||
|
task.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap(),
|
||||||
|
sequenceMetadata.getEndOffsets()
|
||||||
|
);
|
||||||
|
Assert.assertEquals(checkpoints.get(2).keySet(), sequenceMetadata.getExclusiveStartPartitions());
|
||||||
|
Assert.assertFalse(sequenceMetadata.isCheckpointed());
|
||||||
|
}
|
||||||
|
|
||||||
private ListenableFuture<TaskStatus> runTask(final Task task)
|
private ListenableFuture<TaskStatus> runTask(final Task task)
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
|
@ -2522,7 +2594,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||||
private KinesisIndexTask createTask(
|
private KinesisIndexTask createTask(
|
||||||
final String taskId,
|
final String taskId,
|
||||||
final KinesisIndexTaskIOConfig ioConfig
|
final KinesisIndexTaskIOConfig ioConfig
|
||||||
)
|
) throws JsonProcessingException
|
||||||
{
|
{
|
||||||
return createTask(taskId, DATA_SCHEMA, ioConfig, null);
|
return createTask(taskId, DATA_SCHEMA, ioConfig, null);
|
||||||
}
|
}
|
||||||
|
@ -2531,7 +2603,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||||
final String taskId,
|
final String taskId,
|
||||||
final DataSchema dataSchema,
|
final DataSchema dataSchema,
|
||||||
final KinesisIndexTaskIOConfig ioConfig
|
final KinesisIndexTaskIOConfig ioConfig
|
||||||
)
|
) throws JsonProcessingException
|
||||||
{
|
{
|
||||||
return createTask(taskId, dataSchema, ioConfig, null);
|
return createTask(taskId, dataSchema, ioConfig, null);
|
||||||
}
|
}
|
||||||
|
@ -2541,7 +2613,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||||
final DataSchema dataSchema,
|
final DataSchema dataSchema,
|
||||||
final KinesisIndexTaskIOConfig ioConfig,
|
final KinesisIndexTaskIOConfig ioConfig,
|
||||||
@Nullable final Map<String, Object> context
|
@Nullable final Map<String, Object> context
|
||||||
)
|
) throws JsonProcessingException
|
||||||
{
|
{
|
||||||
final KinesisIndexTaskTuningConfig tuningConfig = new KinesisIndexTaskTuningConfig(
|
final KinesisIndexTaskTuningConfig tuningConfig = new KinesisIndexTaskTuningConfig(
|
||||||
maxRowsInMemory,
|
maxRowsInMemory,
|
||||||
|
@ -2578,13 +2650,22 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||||
final KinesisIndexTaskIOConfig ioConfig,
|
final KinesisIndexTaskIOConfig ioConfig,
|
||||||
final KinesisIndexTaskTuningConfig tuningConfig,
|
final KinesisIndexTaskTuningConfig tuningConfig,
|
||||||
@Nullable final Map<String, Object> context
|
@Nullable final Map<String, Object> context
|
||||||
)
|
) throws JsonProcessingException
|
||||||
{
|
{
|
||||||
if (context != null) {
|
if (context != null) {
|
||||||
context.put(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true);
|
context.put(SeekableStreamSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true);
|
||||||
|
|
||||||
|
if (!context.containsKey(SeekableStreamSupervisor.CHECKPOINTS_CTX_KEY)) {
|
||||||
|
final TreeMap<Integer, Map<String, String>> checkpoints = new TreeMap<>();
|
||||||
|
checkpoints.put(0, ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap());
|
||||||
|
final String checkpointsJson = objectMapper
|
||||||
|
.writerFor(KinesisSupervisor.CHECKPOINTS_TYPE_REF)
|
||||||
|
.writeValueAsString(checkpoints);
|
||||||
|
context.put(SeekableStreamSupervisor.CHECKPOINTS_CTX_KEY, checkpointsJson);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
final KinesisIndexTask task = new TestableKinesisIndexTask(
|
return new TestableKinesisIndexTask(
|
||||||
taskId,
|
taskId,
|
||||||
null,
|
null,
|
||||||
cloneDataSchema(dataSchema),
|
cloneDataSchema(dataSchema),
|
||||||
|
@ -2596,8 +2677,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||||
rowIngestionMetersFactory,
|
rowIngestionMetersFactory,
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
|
|
||||||
return task;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static DataSchema cloneDataSchema(final DataSchema dataSchema)
|
private static DataSchema cloneDataSchema(final DataSchema dataSchema)
|
||||||
|
@ -2657,8 +2736,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
|
||||||
objectMapper.registerModule(module);
|
objectMapper.registerModule(module);
|
||||||
}
|
}
|
||||||
final TaskConfig taskConfig = new TaskConfig(
|
final TaskConfig taskConfig = new TaskConfig(
|
||||||
new File(directory, "taskBaseDir").getPath(),
|
new File(directory, "baseDir").getPath(),
|
||||||
null,
|
new File(directory, "baseTaskDir").getPath(),
|
||||||
null,
|
null,
|
||||||
50000,
|
50000,
|
||||||
null,
|
null,
|
||||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.core.type.TypeReference;
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.collect.Iterables;
|
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskUtils;
|
import org.apache.druid.indexing.common.task.IndexTaskUtils;
|
||||||
import org.apache.druid.indexing.common.task.Task;
|
import org.apache.druid.indexing.common.task.Task;
|
||||||
import org.apache.druid.indexing.overlord.CriticalAction;
|
import org.apache.druid.indexing.overlord.CriticalAction;
|
||||||
|
@ -159,8 +158,8 @@ public class SegmentTransactionalInsertAction implements TaskAction<SegmentPubli
|
||||||
@Override
|
@Override
|
||||||
public String toString()
|
public String toString()
|
||||||
{
|
{
|
||||||
return "SegmentInsertAction{" +
|
return "SegmentTransactionalInsertAction{" +
|
||||||
"segments=" + Iterables.transform(segments, DataSegment::getId) +
|
"segments=" + segments +
|
||||||
", startMetadata=" + startMetadata +
|
", startMetadata=" + startMetadata +
|
||||||
", endMetadata=" + endMetadata +
|
", endMetadata=" + endMetadata +
|
||||||
'}';
|
'}';
|
||||||
|
|
|
@ -59,6 +59,7 @@ import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecor
|
||||||
import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
|
import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
|
||||||
import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
|
import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
|
||||||
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
|
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
|
||||||
|
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor;
|
||||||
import org.apache.druid.java.util.common.DateTimes;
|
import org.apache.druid.java.util.common.DateTimes;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
|
@ -239,7 +240,6 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
||||||
resetNextCheckpointTime();
|
resetNextCheckpointTime();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
public TaskStatus run(TaskToolbox toolbox)
|
public TaskStatus run(TaskToolbox toolbox)
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
|
@ -256,57 +256,90 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private TaskStatus runInternal(TaskToolbox toolbox) throws Exception
|
private Set<PartitionIdType> computeExclusiveStartPartitionsForSequence(
|
||||||
|
Map<PartitionIdType, SequenceOffsetType> sequenceStartOffsets
|
||||||
|
)
|
||||||
{
|
{
|
||||||
log.info("SeekableStream indexing task starting up!");
|
if (sequenceStartOffsets.equals(ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap())) {
|
||||||
startTime = DateTimes.nowUtc();
|
return ioConfig.getStartSequenceNumbers().getExclusivePartitions();
|
||||||
status = Status.STARTING;
|
} else {
|
||||||
this.toolbox = toolbox;
|
return isEndOffsetExclusive() ? Collections.emptySet() : sequenceStartOffsets.keySet();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public void setToolbox(TaskToolbox toolbox)
|
||||||
|
{
|
||||||
|
this.toolbox = toolbox;
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public void initializeSequences() throws IOException
|
||||||
|
{
|
||||||
if (!restoreSequences()) {
|
if (!restoreSequences()) {
|
||||||
final TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>> checkpoints = getCheckPointsFromContext(
|
final TreeMap<Integer, Map<PartitionIdType, SequenceOffsetType>> checkpoints = getCheckPointsFromContext(
|
||||||
toolbox,
|
toolbox,
|
||||||
task.getContextValue("checkpoints")
|
task.getContextValue(SeekableStreamSupervisor.CHECKPOINTS_CTX_KEY)
|
||||||
);
|
);
|
||||||
if (checkpoints != null) {
|
if (checkpoints != null) {
|
||||||
boolean exclusive = false;
|
|
||||||
Iterator<Map.Entry<Integer, Map<PartitionIdType, SequenceOffsetType>>> sequenceOffsets = checkpoints.entrySet()
|
Iterator<Map.Entry<Integer, Map<PartitionIdType, SequenceOffsetType>>> sequenceOffsets = checkpoints.entrySet()
|
||||||
.iterator();
|
.iterator();
|
||||||
Map.Entry<Integer, Map<PartitionIdType, SequenceOffsetType>> previous = sequenceOffsets.next();
|
Map.Entry<Integer, Map<PartitionIdType, SequenceOffsetType>> previous = sequenceOffsets.next();
|
||||||
while (sequenceOffsets.hasNext()) {
|
while (sequenceOffsets.hasNext()) {
|
||||||
Map.Entry<Integer, Map<PartitionIdType, SequenceOffsetType>> current = sequenceOffsets.next();
|
Map.Entry<Integer, Map<PartitionIdType, SequenceOffsetType>> current = sequenceOffsets.next();
|
||||||
addSequence(new SequenceMetadata<>(
|
final Set<PartitionIdType> exclusiveStartPartitions = computeExclusiveStartPartitionsForSequence(
|
||||||
previous.getKey(),
|
previous.getValue()
|
||||||
StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()),
|
);
|
||||||
previous.getValue(),
|
addSequence(
|
||||||
current.getValue(),
|
new SequenceMetadata<>(
|
||||||
true,
|
previous.getKey(),
|
||||||
exclusive ? previous.getValue().keySet() : null
|
StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()),
|
||||||
));
|
previous.getValue(),
|
||||||
|
current.getValue(),
|
||||||
|
true,
|
||||||
|
exclusiveStartPartitions
|
||||||
|
)
|
||||||
|
);
|
||||||
previous = current;
|
previous = current;
|
||||||
exclusive = true;
|
|
||||||
}
|
}
|
||||||
addSequence(new SequenceMetadata<>(
|
final Set<PartitionIdType> exclusiveStartPartitions = computeExclusiveStartPartitionsForSequence(
|
||||||
previous.getKey(),
|
previous.getValue()
|
||||||
StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()),
|
);
|
||||||
previous.getValue(),
|
addSequence(
|
||||||
endOffsets,
|
new SequenceMetadata<>(
|
||||||
false,
|
previous.getKey(),
|
||||||
exclusive ? previous.getValue().keySet() : null
|
StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), previous.getKey()),
|
||||||
));
|
previous.getValue(),
|
||||||
|
endOffsets,
|
||||||
|
false,
|
||||||
|
exclusiveStartPartitions
|
||||||
|
)
|
||||||
|
);
|
||||||
} else {
|
} else {
|
||||||
addSequence(new SequenceMetadata<>(
|
addSequence(
|
||||||
0,
|
new SequenceMetadata<>(
|
||||||
StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), 0),
|
0,
|
||||||
ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap(),
|
StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), 0),
|
||||||
endOffsets,
|
ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap(),
|
||||||
false,
|
endOffsets,
|
||||||
null
|
false,
|
||||||
));
|
ioConfig.getStartSequenceNumbers().getExclusivePartitions()
|
||||||
|
)
|
||||||
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
log.info("Starting with sequences: %s", sequences);
|
log.info("Starting with sequences: %s", sequences);
|
||||||
|
}
|
||||||
|
|
||||||
|
private TaskStatus runInternal(TaskToolbox toolbox) throws Exception
|
||||||
|
{
|
||||||
|
log.info("SeekableStream indexing task starting up!");
|
||||||
|
startTime = DateTimes.nowUtc();
|
||||||
|
status = Status.STARTING;
|
||||||
|
|
||||||
|
setToolbox(toolbox);
|
||||||
|
initializeSequences();
|
||||||
|
|
||||||
if (chatHandlerProvider.isPresent()) {
|
if (chatHandlerProvider.isPresent()) {
|
||||||
log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName());
|
log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName());
|
||||||
|
@ -392,10 +425,10 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
||||||
}
|
}
|
||||||
// sequences size can be 0 only when all sequences got published and task stopped before it could finish
|
// sequences size can be 0 only when all sequences got published and task stopped before it could finish
|
||||||
// which is super rare
|
// which is super rare
|
||||||
if (sequences.size() == 0 || sequences.get(sequences.size() - 1).isCheckpointed()) {
|
if (sequences.size() == 0 || getLastSequenceMetadata().isCheckpointed()) {
|
||||||
this.endOffsets.putAll(sequences.size() == 0
|
this.endOffsets.putAll(sequences.size() == 0
|
||||||
? currOffsets
|
? currOffsets
|
||||||
: sequences.get(sequences.size() - 1).getEndOffsets());
|
: getLastSequenceMetadata().getEndOffsets());
|
||||||
log.info("End sequences changed to [%s]", endOffsets);
|
log.info("End sequences changed to [%s]", endOffsets);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -479,7 +512,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
||||||
}
|
}
|
||||||
|
|
||||||
// if stop is requested or task's end sequence is set by call to setEndOffsets method with finish set to true
|
// if stop is requested or task's end sequence is set by call to setEndOffsets method with finish set to true
|
||||||
if (stopRequested.get() || sequences.size() == 0 || sequences.get(sequences.size() - 1).isCheckpointed()) {
|
if (stopRequested.get() || sequences.size() == 0 || getLastSequenceMetadata().isCheckpointed()) {
|
||||||
status = Status.PUBLISHING;
|
status = Status.PUBLISHING;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -628,12 +661,12 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
||||||
}
|
}
|
||||||
|
|
||||||
if (System.currentTimeMillis() > nextCheckpointTime) {
|
if (System.currentTimeMillis() > nextCheckpointTime) {
|
||||||
sequenceToCheckpoint = sequences.get(sequences.size() - 1);
|
sequenceToCheckpoint = getLastSequenceMetadata();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (sequenceToCheckpoint != null && stillReading) {
|
if (sequenceToCheckpoint != null && stillReading) {
|
||||||
Preconditions.checkArgument(
|
Preconditions.checkArgument(
|
||||||
sequences.get(sequences.size() - 1)
|
getLastSequenceMetadata()
|
||||||
.getSequenceName()
|
.getSequenceName()
|
||||||
.equals(sequenceToCheckpoint.getSequenceName()),
|
.equals(sequenceToCheckpoint.getSequenceName()),
|
||||||
"Cannot checkpoint a sequence [%s] which is not the latest one, sequences %s",
|
"Cannot checkpoint a sequence [%s] which is not the latest one, sequences %s",
|
||||||
|
@ -649,7 +682,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
||||||
new SeekableStreamStartSequenceNumbers<>(
|
new SeekableStreamStartSequenceNumbers<>(
|
||||||
stream,
|
stream,
|
||||||
sequenceToCheckpoint.getStartOffsets(),
|
sequenceToCheckpoint.getStartOffsets(),
|
||||||
ioConfig.getStartSequenceNumbers().getExclusivePartitions()
|
sequenceToCheckpoint.getExclusiveStartPartitions()
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
createDataSourceMetadata(
|
createDataSourceMetadata(
|
||||||
|
@ -1060,7 +1093,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
||||||
final SequenceOffsetType startOffset = entry.getValue();
|
final SequenceOffsetType startOffset = entry.getValue();
|
||||||
|
|
||||||
if (!sequences.isEmpty()) {
|
if (!sequences.isEmpty()) {
|
||||||
final SequenceOffsetType priorOffset = sequences.get(sequences.size() - 1).endOffsets.get(partition);
|
final SequenceOffsetType priorOffset = getLastSequenceMetadata().endOffsets.get(partition);
|
||||||
|
|
||||||
if (!startOffset.equals(priorOffset)) {
|
if (!startOffset.equals(priorOffset)) {
|
||||||
throw new ISE(
|
throw new ISE(
|
||||||
|
@ -1072,9 +1105,26 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (!isEndOffsetExclusive() && !sequences.isEmpty()) {
|
||||||
|
final SequenceMetadata<PartitionIdType, SequenceOffsetType> lastMetadata = getLastSequenceMetadata();
|
||||||
|
if (!lastMetadata.endOffsets.keySet().equals(sequenceMetadata.getExclusiveStartPartitions())) {
|
||||||
|
throw new ISE(
|
||||||
|
"Exclusive start partitions[%s] for new sequence don't match to the prior offset[%s]",
|
||||||
|
sequenceMetadata.getExclusiveStartPartitions(),
|
||||||
|
lastMetadata
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// Actually do the add.
|
// Actually do the add.
|
||||||
sequences.add(sequenceMetadata);
|
sequences.add(sequenceMetadata);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private SequenceMetadata<PartitionIdType, SequenceOffsetType> getLastSequenceMetadata()
|
||||||
|
{
|
||||||
|
Preconditions.checkState(!sequences.isEmpty(), "Empty sequences");
|
||||||
|
return sequences.get(sequences.size() - 1);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns true if the given record has already been read, based on lastReadOffsets.
|
* Returns true if the given record has already been read, based on lastReadOffsets.
|
||||||
|
@ -1453,7 +1503,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
||||||
// and after acquiring pauseLock to correctly guard against duplicate requests
|
// and after acquiring pauseLock to correctly guard against duplicate requests
|
||||||
Preconditions.checkState(sequenceNumbers.size() > 0, "WTH?! No Sequences found to set end sequences");
|
Preconditions.checkState(sequenceNumbers.size() > 0, "WTH?! No Sequences found to set end sequences");
|
||||||
|
|
||||||
final SequenceMetadata<PartitionIdType, SequenceOffsetType> latestSequence = sequences.get(sequences.size() - 1);
|
final SequenceMetadata<PartitionIdType, SequenceOffsetType> latestSequence = getLastSequenceMetadata();
|
||||||
final Set<PartitionIdType> exclusiveStartPartitions;
|
final Set<PartitionIdType> exclusiveStartPartitions;
|
||||||
|
|
||||||
if (isEndOffsetExclusive()) {
|
if (isEndOffsetExclusive()) {
|
||||||
|
@ -1545,6 +1595,12 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
||||||
nextCheckpointTime = DateTimes.nowUtc().plus(tuningConfig.getIntermediateHandoffPeriod()).getMillis();
|
nextCheckpointTime = DateTimes.nowUtc().plus(tuningConfig.getIntermediateHandoffPeriod()).getMillis();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public CopyOnWriteArrayList<SequenceMetadata<PartitionIdType, SequenceOffsetType>> getSequences()
|
||||||
|
{
|
||||||
|
return sequences;
|
||||||
|
}
|
||||||
|
|
||||||
@GET
|
@GET
|
||||||
@Path("/checkpoints")
|
@Path("/checkpoints")
|
||||||
@Produces(MediaType.APPLICATION_JSON)
|
@Produces(MediaType.APPLICATION_JSON)
|
||||||
|
|
|
@ -216,11 +216,12 @@ public class SequenceMetadata<PartitionIdType, SequenceOffsetType>
|
||||||
lock.lock();
|
lock.lock();
|
||||||
try {
|
try {
|
||||||
return "SequenceMetadata{" +
|
return "SequenceMetadata{" +
|
||||||
"sequenceName='" + sequenceName + '\'' +
|
"sequenceId=" + sequenceId +
|
||||||
", sequenceId=" + sequenceId +
|
", sequenceName='" + sequenceName + '\'' +
|
||||||
", startOffsets=" + startOffsets +
|
|
||||||
", endOffsets=" + endOffsets +
|
|
||||||
", assignments=" + assignments +
|
", assignments=" + assignments +
|
||||||
|
", startOffsets=" + startOffsets +
|
||||||
|
", exclusiveStartPartitions=" + exclusiveStartPartitions +
|
||||||
|
", endOffsets=" + endOffsets +
|
||||||
", sentinel=" + sentinel +
|
", sentinel=" + sentinel +
|
||||||
", checkpointed=" + checkpointed +
|
", checkpointed=" + checkpointed +
|
||||||
'}';
|
'}';
|
||||||
|
|
|
@ -121,6 +121,7 @@ import java.util.stream.Stream;
|
||||||
public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetType> implements Supervisor
|
public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetType> implements Supervisor
|
||||||
{
|
{
|
||||||
public static final String IS_INCREMENTAL_HANDOFF_SUPPORTED = "IS_INCREMENTAL_HANDOFF_SUPPORTED";
|
public static final String IS_INCREMENTAL_HANDOFF_SUPPORTED = "IS_INCREMENTAL_HANDOFF_SUPPORTED";
|
||||||
|
public static final String CHECKPOINTS_CTX_KEY = "checkpoints";
|
||||||
|
|
||||||
private static final long MAX_RUN_FREQUENCY_MILLIS = 1000;
|
private static final long MAX_RUN_FREQUENCY_MILLIS = 1000;
|
||||||
private static final long MINIMUM_FUTURE_TIMEOUT_IN_SECONDS = 120;
|
private static final long MINIMUM_FUTURE_TIMEOUT_IN_SECONDS = 120;
|
||||||
|
@ -163,7 +164,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
||||||
ImmutableMap<PartitionIdType, SequenceOffsetType> startingSequences,
|
ImmutableMap<PartitionIdType, SequenceOffsetType> startingSequences,
|
||||||
Optional<DateTime> minimumMessageTime,
|
Optional<DateTime> minimumMessageTime,
|
||||||
Optional<DateTime> maximumMessageTime,
|
Optional<DateTime> maximumMessageTime,
|
||||||
Set<PartitionIdType> exclusiveStartSequenceNumberPartitions
|
@Nullable Set<PartitionIdType> exclusiveStartSequenceNumberPartitions
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.groupId = groupId;
|
this.groupId = groupId;
|
||||||
|
@ -173,7 +174,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
||||||
this.checkpointSequences.put(0, startingSequences);
|
this.checkpointSequences.put(0, startingSequences);
|
||||||
this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions != null
|
this.exclusiveStartSequenceNumberPartitions = exclusiveStartSequenceNumberPartitions != null
|
||||||
? exclusiveStartSequenceNumberPartitions
|
? exclusiveStartSequenceNumberPartitions
|
||||||
: new HashSet<>();
|
: Collections.emptySet();
|
||||||
this.baseSequenceName = generateSequenceName(startingSequences, minimumMessageTime, maximumMessageTime);
|
this.baseSequenceName = generateSequenceName(startingSequences, minimumMessageTime, maximumMessageTime);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2347,7 +2348,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
||||||
if (!getNotSetMarker().equals(sequence)) {
|
if (!getNotSetMarker().equals(sequence)) {
|
||||||
// if we are given a startingOffset (set by a previous task group which is pending completion) then use it
|
// if we are given a startingOffset (set by a previous task group which is pending completion) then use it
|
||||||
if (!isEndOfShard(sequence)) {
|
if (!isEndOfShard(sequence)) {
|
||||||
builder.put(partition, makeSequenceNumber(sequence, useExclusiveStartSequenceNumberForStartSequence()));
|
builder.put(partition, makeSequenceNumber(sequence, useExclusiveStartSequenceNumberForNonFirstSequence()));
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
// if we don't have a startingOffset (first run or we had some previous failures and reset the sequences) then
|
// if we don't have a startingOffset (first run or we had some previous failures and reset the sequences) then
|
||||||
|
@ -2393,7 +2394,7 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return makeSequenceNumber(sequence, useExclusiveStartSequenceNumberForStartSequence());
|
return makeSequenceNumber(sequence, useExclusiveStartSequenceNumberForNonFirstSequence());
|
||||||
} else {
|
} else {
|
||||||
boolean useEarliestSequenceNumber = ioConfig.isUseEarliestSequenceNumber();
|
boolean useEarliestSequenceNumber = ioConfig.isUseEarliestSequenceNumber();
|
||||||
if (subsequentlyDiscoveredPartitions.contains(partition)) {
|
if (subsequentlyDiscoveredPartitions.contains(partition)) {
|
||||||
|
@ -2469,7 +2470,9 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
||||||
for (PartitionIdType partition : startPartitions.keySet()) {
|
for (PartitionIdType partition : startPartitions.keySet()) {
|
||||||
endPartitions.put(partition, getEndOfPartitionMarker());
|
endPartitions.put(partition, getEndOfPartitionMarker());
|
||||||
}
|
}
|
||||||
Set<PartitionIdType> exclusiveStartSequenceNumberPartitions = activelyReadingTaskGroups.get(groupId).exclusiveStartSequenceNumberPartitions;
|
Set<PartitionIdType> exclusiveStartSequenceNumberPartitions = activelyReadingTaskGroups
|
||||||
|
.get(groupId)
|
||||||
|
.exclusiveStartSequenceNumberPartitions;
|
||||||
|
|
||||||
DateTime minimumMessageTime = activelyReadingTaskGroups.get(groupId).minimumMessageTime.orNull();
|
DateTime minimumMessageTime = activelyReadingTaskGroups.get(groupId).minimumMessageTime.orNull();
|
||||||
DateTime maximumMessageTime = activelyReadingTaskGroups.get(groupId).maximumMessageTime.orNull();
|
DateTime maximumMessageTime = activelyReadingTaskGroups.get(groupId).maximumMessageTime.orNull();
|
||||||
|
@ -2660,6 +2663,16 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected Map<String, Object> createBaseTaskContexts()
|
||||||
|
{
|
||||||
|
final Map<String, Object> contexts = new HashMap<>();
|
||||||
|
contexts.put(IS_INCREMENTAL_HANDOFF_SUPPORTED, true);
|
||||||
|
if (spec.getContext() != null) {
|
||||||
|
contexts.putAll(spec.getContext());
|
||||||
|
}
|
||||||
|
return contexts;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* creates a specific task IOConfig instance for Kafka/Kinesis
|
* creates a specific task IOConfig instance for Kafka/Kinesis
|
||||||
*
|
*
|
||||||
|
@ -2827,5 +2840,5 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
||||||
* For example, in Kinesis, the start offsets are inclusive for the first sequence, but exclusive for following
|
* For example, in Kinesis, the start offsets are inclusive for the first sequence, but exclusive for following
|
||||||
* sequences. In Kafka, start offsets are always inclusive.
|
* sequences. In Kafka, start offsets are always inclusive.
|
||||||
*/
|
*/
|
||||||
protected abstract boolean useExclusiveStartSequenceNumberForStartSequence();
|
protected abstract boolean useExclusiveStartSequenceNumberForNonFirstSequence();
|
||||||
}
|
}
|
||||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig;
|
import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -51,6 +52,7 @@ public abstract class SeekableStreamSupervisorSpec implements SupervisorSpec
|
||||||
private final DataSchema dataSchema;
|
private final DataSchema dataSchema;
|
||||||
private final SeekableStreamSupervisorTuningConfig tuningConfig;
|
private final SeekableStreamSupervisorTuningConfig tuningConfig;
|
||||||
private final SeekableStreamSupervisorIOConfig ioConfig;
|
private final SeekableStreamSupervisorIOConfig ioConfig;
|
||||||
|
@Nullable
|
||||||
private final Map<String, Object> context;
|
private final Map<String, Object> context;
|
||||||
protected final ServiceEmitter emitter;
|
protected final ServiceEmitter emitter;
|
||||||
protected final DruidMonitorSchedulerConfig monitorSchedulerConfig;
|
protected final DruidMonitorSchedulerConfig monitorSchedulerConfig;
|
||||||
|
@ -61,7 +63,7 @@ public abstract class SeekableStreamSupervisorSpec implements SupervisorSpec
|
||||||
@JsonProperty("dataSchema") DataSchema dataSchema,
|
@JsonProperty("dataSchema") DataSchema dataSchema,
|
||||||
@JsonProperty("tuningConfig") SeekableStreamSupervisorTuningConfig tuningConfig,
|
@JsonProperty("tuningConfig") SeekableStreamSupervisorTuningConfig tuningConfig,
|
||||||
@JsonProperty("ioConfig") SeekableStreamSupervisorIOConfig ioConfig,
|
@JsonProperty("ioConfig") SeekableStreamSupervisorIOConfig ioConfig,
|
||||||
@JsonProperty("context") Map<String, Object> context,
|
@JsonProperty("context") @Nullable Map<String, Object> context,
|
||||||
@JsonProperty("suspended") Boolean suspended,
|
@JsonProperty("suspended") Boolean suspended,
|
||||||
@JacksonInject TaskStorage taskStorage,
|
@JacksonInject TaskStorage taskStorage,
|
||||||
@JacksonInject TaskMaster taskMaster,
|
@JacksonInject TaskMaster taskMaster,
|
||||||
|
@ -107,6 +109,7 @@ public abstract class SeekableStreamSupervisorSpec implements SupervisorSpec
|
||||||
return ioConfig;
|
return ioConfig;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public Map<String, Object> getContext()
|
public Map<String, Object> getContext()
|
||||||
{
|
{
|
||||||
|
|
|
@ -891,8 +891,11 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
||||||
|
|
||||||
if (!startMetadataMatchesExisting) {
|
if (!startMetadataMatchesExisting) {
|
||||||
// Not in the desired start state.
|
// Not in the desired start state.
|
||||||
log.info("Not updating metadata, existing state is not the expected start state.");
|
log.error(
|
||||||
log.debug("Existing database state [%s], request's start metadata [%s]", oldCommitMetadataFromDb, startMetadata);
|
"Not updating metadata, existing state[%s] in metadata store doesn't match to the new start state[%s].",
|
||||||
|
oldCommitMetadataBytesFromDb,
|
||||||
|
startMetadata
|
||||||
|
);
|
||||||
return DataSourceMetadataUpdateResult.FAILURE;
|
return DataSourceMetadataUpdateResult.FAILURE;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue