mirror of https://github.com/apache/druid.git
Fix NPE while handling CheckpointNotice in KafkaSupervisor (#5996)
* Fix NPE while handling CheckpointNotice * fix code style * Fix test * fix test * add a log for creating a new taskGroup * fix backward compatibility in KafkaIOConfig
This commit is contained in:
parent
31c2179fe1
commit
c48aa74a30
|
@ -53,7 +53,6 @@ import io.druid.timeline.DataSegment;
|
||||||
import org.joda.time.Duration;
|
import org.joda.time.Duration;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -240,11 +239,7 @@ public class MaterializedViewSupervisor implements Supervisor
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void checkpoint(
|
public void checkpoint(int taskGroupId, DataSourceMetadata previousCheckPoint, DataSourceMetadata currentCheckPoint)
|
||||||
@Nullable String sequenceName,
|
|
||||||
@Nullable DataSourceMetadata previousCheckPoint,
|
|
||||||
@Nullable DataSourceMetadata currentCheckPoint
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
// do nothing
|
// do nothing
|
||||||
}
|
}
|
||||||
|
|
|
@ -600,12 +600,13 @@ public class IncrementalPublishingKafkaIndexTaskRunner implements KafkaIndexTask
|
||||||
sequences
|
sequences
|
||||||
);
|
);
|
||||||
requestPause();
|
requestPause();
|
||||||
if (!toolbox.getTaskActionClient().submit(new CheckPointDataSourceMetadataAction(
|
final CheckPointDataSourceMetadataAction checkpointAction = new CheckPointDataSourceMetadataAction(
|
||||||
task.getDataSource(),
|
task.getDataSource(),
|
||||||
ioConfig.getBaseSequenceName(),
|
ioConfig.getTaskGroupId(),
|
||||||
new KafkaDataSourceMetadata(new KafkaPartitions(topic, sequenceToCheckpoint.getStartOffsets())),
|
new KafkaDataSourceMetadata(new KafkaPartitions(topic, sequenceToCheckpoint.getStartOffsets())),
|
||||||
new KafkaDataSourceMetadata(new KafkaPartitions(topic, nextOffsets))
|
new KafkaDataSourceMetadata(new KafkaPartitions(topic, nextOffsets))
|
||||||
))) {
|
);
|
||||||
|
if (!toolbox.getTaskActionClient().submit(checkpointAction)) {
|
||||||
throw new ISE("Checkpoint request with offsets [%s] failed, dying", nextOffsets);
|
throw new ISE("Checkpoint request with offsets [%s] failed, dying", nextOffsets);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -26,6 +26,7 @@ import com.google.common.base.Preconditions;
|
||||||
import io.druid.segment.indexing.IOConfig;
|
import io.druid.segment.indexing.IOConfig;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
public class KafkaIOConfig implements IOConfig
|
public class KafkaIOConfig implements IOConfig
|
||||||
|
@ -33,6 +34,8 @@ public class KafkaIOConfig implements IOConfig
|
||||||
private static final boolean DEFAULT_USE_TRANSACTION = true;
|
private static final boolean DEFAULT_USE_TRANSACTION = true;
|
||||||
private static final boolean DEFAULT_SKIP_OFFSET_GAPS = false;
|
private static final boolean DEFAULT_SKIP_OFFSET_GAPS = false;
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
private final Integer taskGroupId;
|
||||||
private final String baseSequenceName;
|
private final String baseSequenceName;
|
||||||
private final KafkaPartitions startPartitions;
|
private final KafkaPartitions startPartitions;
|
||||||
private final KafkaPartitions endPartitions;
|
private final KafkaPartitions endPartitions;
|
||||||
|
@ -44,6 +47,7 @@ public class KafkaIOConfig implements IOConfig
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public KafkaIOConfig(
|
public KafkaIOConfig(
|
||||||
|
@JsonProperty("taskGroupId") @Nullable Integer taskGroupId, // can be null for backward compabitility
|
||||||
@JsonProperty("baseSequenceName") String baseSequenceName,
|
@JsonProperty("baseSequenceName") String baseSequenceName,
|
||||||
@JsonProperty("startPartitions") KafkaPartitions startPartitions,
|
@JsonProperty("startPartitions") KafkaPartitions startPartitions,
|
||||||
@JsonProperty("endPartitions") KafkaPartitions endPartitions,
|
@JsonProperty("endPartitions") KafkaPartitions endPartitions,
|
||||||
|
@ -54,6 +58,7 @@ public class KafkaIOConfig implements IOConfig
|
||||||
@JsonProperty("skipOffsetGaps") Boolean skipOffsetGaps
|
@JsonProperty("skipOffsetGaps") Boolean skipOffsetGaps
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
this.taskGroupId = taskGroupId;
|
||||||
this.baseSequenceName = Preconditions.checkNotNull(baseSequenceName, "baseSequenceName");
|
this.baseSequenceName = Preconditions.checkNotNull(baseSequenceName, "baseSequenceName");
|
||||||
this.startPartitions = Preconditions.checkNotNull(startPartitions, "startPartitions");
|
this.startPartitions = Preconditions.checkNotNull(startPartitions, "startPartitions");
|
||||||
this.endPartitions = Preconditions.checkNotNull(endPartitions, "endPartitions");
|
this.endPartitions = Preconditions.checkNotNull(endPartitions, "endPartitions");
|
||||||
|
@ -83,6 +88,13 @@ public class KafkaIOConfig implements IOConfig
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@JsonProperty
|
||||||
|
public Integer getTaskGroupId()
|
||||||
|
{
|
||||||
|
return taskGroupId;
|
||||||
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public String getBaseSequenceName()
|
public String getBaseSequenceName()
|
||||||
{
|
{
|
||||||
|
@ -135,7 +147,8 @@ public class KafkaIOConfig implements IOConfig
|
||||||
public String toString()
|
public String toString()
|
||||||
{
|
{
|
||||||
return "KafkaIOConfig{" +
|
return "KafkaIOConfig{" +
|
||||||
"baseSequenceName='" + baseSequenceName + '\'' +
|
"taskGroupId=" + taskGroupId +
|
||||||
|
", baseSequenceName='" + baseSequenceName + '\'' +
|
||||||
", startPartitions=" + startPartitions +
|
", startPartitions=" + startPartitions +
|
||||||
", endPartitions=" + endPartitions +
|
", endPartitions=" + endPartitions +
|
||||||
", consumerProperties=" + consumerProperties +
|
", consumerProperties=" + consumerProperties +
|
||||||
|
|
|
@ -143,8 +143,7 @@ public class KafkaSupervisor implements Supervisor
|
||||||
* time, there should only be up to a maximum of [taskCount] actively-reading task groups (tracked in the [taskGroups]
|
* time, there should only be up to a maximum of [taskCount] actively-reading task groups (tracked in the [taskGroups]
|
||||||
* map) + zero or more pending-completion task groups (tracked in [pendingCompletionTaskGroups]).
|
* map) + zero or more pending-completion task groups (tracked in [pendingCompletionTaskGroups]).
|
||||||
*/
|
*/
|
||||||
@VisibleForTesting
|
private static class TaskGroup
|
||||||
static class TaskGroup
|
|
||||||
{
|
{
|
||||||
// This specifies the partitions and starting offsets for this task group. It is set on group creation from the data
|
// This specifies the partitions and starting offsets for this task group. It is set on group creation from the data
|
||||||
// in [partitionGroups] and never changes during the lifetime of this task group, which will live until a task in
|
// in [partitionGroups] and never changes during the lifetime of this task group, which will live until a task in
|
||||||
|
@ -159,7 +158,7 @@ public class KafkaSupervisor implements Supervisor
|
||||||
DateTime completionTimeout; // is set after signalTasksToFinish(); if not done by timeout, take corrective action
|
DateTime completionTimeout; // is set after signalTasksToFinish(); if not done by timeout, take corrective action
|
||||||
final TreeMap<Integer, Map<Integer, Long>> sequenceOffsets = new TreeMap<>();
|
final TreeMap<Integer, Map<Integer, Long>> sequenceOffsets = new TreeMap<>();
|
||||||
|
|
||||||
public TaskGroup(
|
TaskGroup(
|
||||||
ImmutableMap<Integer, Long> partitionOffsets,
|
ImmutableMap<Integer, Long> partitionOffsets,
|
||||||
Optional<DateTime> minimumMessageTime,
|
Optional<DateTime> minimumMessageTime,
|
||||||
Optional<DateTime> maximumMessageTime
|
Optional<DateTime> maximumMessageTime
|
||||||
|
@ -171,7 +170,7 @@ public class KafkaSupervisor implements Supervisor
|
||||||
this.sequenceOffsets.put(0, partitionOffsets);
|
this.sequenceOffsets.put(0, partitionOffsets);
|
||||||
}
|
}
|
||||||
|
|
||||||
public int addNewCheckpoint(Map<Integer, Long> checkpoint)
|
int addNewCheckpoint(Map<Integer, Long> checkpoint)
|
||||||
{
|
{
|
||||||
sequenceOffsets.put(sequenceOffsets.lastKey() + 1, checkpoint);
|
sequenceOffsets.put(sequenceOffsets.lastKey() + 1, checkpoint);
|
||||||
return sequenceOffsets.lastKey();
|
return sequenceOffsets.lastKey();
|
||||||
|
@ -212,9 +211,6 @@ public class KafkaSupervisor implements Supervisor
|
||||||
private final ConcurrentHashMap<Integer, ConcurrentHashMap<Integer, Long>> partitionGroups = new ConcurrentHashMap<>();
|
private final ConcurrentHashMap<Integer, ConcurrentHashMap<Integer, Long>> partitionGroups = new ConcurrentHashMap<>();
|
||||||
// --------------------------------------------------------
|
// --------------------------------------------------------
|
||||||
|
|
||||||
// BaseSequenceName -> TaskGroup
|
|
||||||
private final ConcurrentHashMap<String, TaskGroup> sequenceTaskGroup = new ConcurrentHashMap<>();
|
|
||||||
|
|
||||||
private final TaskStorage taskStorage;
|
private final TaskStorage taskStorage;
|
||||||
private final TaskMaster taskMaster;
|
private final TaskMaster taskMaster;
|
||||||
private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
|
private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
|
||||||
|
@ -513,13 +509,9 @@ public class KafkaSupervisor implements Supervisor
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void checkpoint(
|
public void checkpoint(int taskGroupId, DataSourceMetadata previousCheckpoint, DataSourceMetadata currentCheckpoint)
|
||||||
String sequenceName,
|
|
||||||
DataSourceMetadata previousCheckpoint,
|
|
||||||
DataSourceMetadata currentCheckpoint
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
Preconditions.checkNotNull(sequenceName, "Cannot checkpoint without a sequence name");
|
Preconditions.checkNotNull(previousCheckpoint, "previousCheckpoint");
|
||||||
Preconditions.checkNotNull(currentCheckpoint, "current checkpoint cannot be null");
|
Preconditions.checkNotNull(currentCheckpoint, "current checkpoint cannot be null");
|
||||||
Preconditions.checkArgument(
|
Preconditions.checkArgument(
|
||||||
ioConfig.getTopic()
|
ioConfig.getTopic()
|
||||||
|
@ -530,12 +522,14 @@ public class KafkaSupervisor implements Supervisor
|
||||||
((KafkaDataSourceMetadata) currentCheckpoint).getKafkaPartitions().getTopic()
|
((KafkaDataSourceMetadata) currentCheckpoint).getKafkaPartitions().getTopic()
|
||||||
);
|
);
|
||||||
|
|
||||||
log.info("Checkpointing [%s] for sequence [%s]", currentCheckpoint, sequenceName);
|
log.info("Checkpointing [%s] for taskGroup [%s]", currentCheckpoint, taskGroupId);
|
||||||
notices.add(new CheckpointNotice(
|
notices.add(
|
||||||
sequenceName,
|
new CheckpointNotice(
|
||||||
(KafkaDataSourceMetadata) previousCheckpoint,
|
taskGroupId,
|
||||||
(KafkaDataSourceMetadata) currentCheckpoint
|
(KafkaDataSourceMetadata) previousCheckpoint,
|
||||||
));
|
(KafkaDataSourceMetadata) currentCheckpoint
|
||||||
|
)
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void possiblyRegisterListener()
|
public void possiblyRegisterListener()
|
||||||
|
@ -637,17 +631,17 @@ public class KafkaSupervisor implements Supervisor
|
||||||
|
|
||||||
private class CheckpointNotice implements Notice
|
private class CheckpointNotice implements Notice
|
||||||
{
|
{
|
||||||
final String sequenceName;
|
final int taskGroupId;
|
||||||
final KafkaDataSourceMetadata previousCheckpoint;
|
final KafkaDataSourceMetadata previousCheckpoint;
|
||||||
final KafkaDataSourceMetadata currentCheckpoint;
|
final KafkaDataSourceMetadata currentCheckpoint;
|
||||||
|
|
||||||
CheckpointNotice(
|
CheckpointNotice(
|
||||||
String sequenceName,
|
int taskGroupId,
|
||||||
KafkaDataSourceMetadata previousCheckpoint,
|
KafkaDataSourceMetadata previousCheckpoint,
|
||||||
KafkaDataSourceMetadata currentCheckpoint
|
KafkaDataSourceMetadata currentCheckpoint
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.sequenceName = sequenceName;
|
this.taskGroupId = taskGroupId;
|
||||||
this.previousCheckpoint = previousCheckpoint;
|
this.previousCheckpoint = previousCheckpoint;
|
||||||
this.currentCheckpoint = currentCheckpoint;
|
this.currentCheckpoint = currentCheckpoint;
|
||||||
}
|
}
|
||||||
|
@ -658,17 +652,12 @@ public class KafkaSupervisor implements Supervisor
|
||||||
// check for consistency
|
// check for consistency
|
||||||
// if already received request for this sequenceName and dataSourceMetadata combination then return
|
// if already received request for this sequenceName and dataSourceMetadata combination then return
|
||||||
|
|
||||||
Preconditions.checkNotNull(
|
final TaskGroup taskGroup = taskGroups.get(taskGroupId);
|
||||||
sequenceTaskGroup.get(sequenceName),
|
|
||||||
"WTH?! cannot find task group for this sequence [%s], sequencesTaskGroup map [%s], taskGroups [%s]",
|
|
||||||
sequenceName,
|
|
||||||
sequenceTaskGroup,
|
|
||||||
taskGroups
|
|
||||||
);
|
|
||||||
final TreeMap<Integer, Map<Integer, Long>> checkpoints = sequenceTaskGroup.get(sequenceName).sequenceOffsets;
|
|
||||||
|
|
||||||
// check validity of previousCheckpoint if it is not null
|
if (isValidTaskGroup(taskGroup)) {
|
||||||
if (previousCheckpoint != null) {
|
final TreeMap<Integer, Map<Integer, Long>> checkpoints = taskGroup.sequenceOffsets;
|
||||||
|
|
||||||
|
// check validity of previousCheckpoint
|
||||||
int index = checkpoints.size();
|
int index = checkpoints.size();
|
||||||
for (int sequenceId : checkpoints.descendingKeySet()) {
|
for (int sequenceId : checkpoints.descendingKeySet()) {
|
||||||
Map<Integer, Long> checkpoint = checkpoints.get(sequenceId);
|
Map<Integer, Long> checkpoint = checkpoints.get(sequenceId);
|
||||||
|
@ -685,26 +674,39 @@ public class KafkaSupervisor implements Supervisor
|
||||||
log.info("Already checkpointed with offsets [%s]", checkpoints.lastEntry().getValue());
|
log.info("Already checkpointed with offsets [%s]", checkpoints.lastEntry().getValue());
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
} else {
|
final int taskGroupId = getTaskGroupIdForPartition(
|
||||||
// There cannot be more than one checkpoint when previous checkpoint is null
|
currentCheckpoint.getKafkaPartitions()
|
||||||
// as when the task starts they are sent existing checkpoints
|
.getPartitionOffsetMap()
|
||||||
Preconditions.checkState(
|
.keySet()
|
||||||
checkpoints.size() <= 1,
|
.iterator()
|
||||||
"Got checkpoint request with null as previous check point, however found more than one checkpoints"
|
.next()
|
||||||
);
|
);
|
||||||
if (checkpoints.size() == 1) {
|
final Map<Integer, Long> newCheckpoint = checkpointTaskGroup(taskGroupId, false).get();
|
||||||
log.info("Already checkpointed with dataSourceMetadata [%s]", checkpoints.get(0));
|
taskGroups.get(taskGroupId).addNewCheckpoint(newCheckpoint);
|
||||||
return;
|
log.info("Handled checkpoint notice, new checkpoint is [%s] for taskGroup [%s]", newCheckpoint, taskGroupId);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean isValidTaskGroup(@Nullable TaskGroup taskGroup)
|
||||||
|
{
|
||||||
|
if (taskGroup == null) {
|
||||||
|
// taskGroup might be in pendingCompletionTaskGroups or partitionGroups
|
||||||
|
if (pendingCompletionTaskGroups.containsKey(taskGroupId)) {
|
||||||
|
log.warn(
|
||||||
|
"Ignoring checkpoint request because taskGroup[%d] has already stopped indexing and is waiting for "
|
||||||
|
+ "publishing segments",
|
||||||
|
taskGroupId
|
||||||
|
);
|
||||||
|
return false;
|
||||||
|
} else if (partitionGroups.containsKey(taskGroupId)) {
|
||||||
|
log.warn("Ignoring checkpoint request because taskGroup[%d] is inactive", taskGroupId);
|
||||||
|
return false;
|
||||||
|
} else {
|
||||||
|
throw new ISE("WTH?! cannot find taskGroup [%s] among all taskGroups [%s]", taskGroupId, taskGroups);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
final int taskGroupId = getTaskGroupIdForPartition(currentCheckpoint.getKafkaPartitions()
|
|
||||||
.getPartitionOffsetMap()
|
return true;
|
||||||
.keySet()
|
|
||||||
.iterator()
|
|
||||||
.next());
|
|
||||||
final Map<Integer, Long> newCheckpoint = checkpointTaskGroup(taskGroupId, false).get();
|
|
||||||
sequenceTaskGroup.get(sequenceName).addNewCheckpoint(newCheckpoint);
|
|
||||||
log.info("Handled checkpoint notice, new checkpoint is [%s] for sequence [%s]", newCheckpoint, sequenceName);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -718,7 +720,6 @@ public class KafkaSupervisor implements Supervisor
|
||||||
taskGroups.values().forEach(this::killTasksInGroup);
|
taskGroups.values().forEach(this::killTasksInGroup);
|
||||||
taskGroups.clear();
|
taskGroups.clear();
|
||||||
partitionGroups.clear();
|
partitionGroups.clear();
|
||||||
sequenceTaskGroup.clear();
|
|
||||||
} else if (!(dataSourceMetadata instanceof KafkaDataSourceMetadata)) {
|
} else if (!(dataSourceMetadata instanceof KafkaDataSourceMetadata)) {
|
||||||
throw new IAE("Expected KafkaDataSourceMetadata but found instance of [%s]", dataSourceMetadata.getClass());
|
throw new IAE("Expected KafkaDataSourceMetadata but found instance of [%s]", dataSourceMetadata.getClass());
|
||||||
} else {
|
} else {
|
||||||
|
@ -778,8 +779,7 @@ public class KafkaSupervisor implements Supervisor
|
||||||
resetKafkaMetadata.getKafkaPartitions().getPartitionOffsetMap().keySet().forEach(partition -> {
|
resetKafkaMetadata.getKafkaPartitions().getPartitionOffsetMap().keySet().forEach(partition -> {
|
||||||
final int groupId = getTaskGroupIdForPartition(partition);
|
final int groupId = getTaskGroupIdForPartition(partition);
|
||||||
killTaskGroupForPartitions(ImmutableSet.of(partition));
|
killTaskGroupForPartitions(ImmutableSet.of(partition));
|
||||||
final TaskGroup removedGroup = taskGroups.remove(groupId);
|
taskGroups.remove(groupId);
|
||||||
sequenceTaskGroup.remove(generateSequenceName(removedGroup));
|
|
||||||
partitionGroups.get(groupId).replaceAll((partitionId, offset) -> NOT_SET);
|
partitionGroups.get(groupId).replaceAll((partitionId, offset) -> NOT_SET);
|
||||||
});
|
});
|
||||||
} else {
|
} else {
|
||||||
|
@ -955,9 +955,10 @@ public class KafkaSupervisor implements Supervisor
|
||||||
for (int partition = 0; partition < numPartitions; partition++) {
|
for (int partition = 0; partition < numPartitions; partition++) {
|
||||||
int taskGroupId = getTaskGroupIdForPartition(partition);
|
int taskGroupId = getTaskGroupIdForPartition(partition);
|
||||||
|
|
||||||
partitionGroups.putIfAbsent(taskGroupId, new ConcurrentHashMap<Integer, Long>());
|
ConcurrentHashMap<Integer, Long> partitionMap = partitionGroups.computeIfAbsent(
|
||||||
|
taskGroupId,
|
||||||
ConcurrentHashMap<Integer, Long> partitionMap = partitionGroups.get(taskGroupId);
|
k -> new ConcurrentHashMap<>()
|
||||||
|
);
|
||||||
|
|
||||||
// The starting offset for a new partition in [partitionGroups] is initially set to NOT_SET; when a new task group
|
// The starting offset for a new partition in [partitionGroups] is initially set to NOT_SET; when a new task group
|
||||||
// is created and is assigned partitions, if the offset in [partitionGroups] is NOT_SET it will take the starting
|
// is created and is assigned partitions, if the offset in [partitionGroups] is NOT_SET it will take the starting
|
||||||
|
@ -1087,23 +1088,21 @@ public class KafkaSupervisor implements Supervisor
|
||||||
}
|
}
|
||||||
return false;
|
return false;
|
||||||
} else {
|
} else {
|
||||||
final TaskGroup taskGroup = new TaskGroup(
|
final TaskGroup taskGroup = taskGroups.computeIfAbsent(
|
||||||
ImmutableMap.copyOf(
|
|
||||||
kafkaTask.getIOConfig()
|
|
||||||
.getStartPartitions()
|
|
||||||
.getPartitionOffsetMap()
|
|
||||||
), kafkaTask.getIOConfig().getMinimumMessageTime(),
|
|
||||||
kafkaTask.getIOConfig().getMaximumMessageTime()
|
|
||||||
);
|
|
||||||
if (taskGroups.putIfAbsent(
|
|
||||||
taskGroupId,
|
taskGroupId,
|
||||||
taskGroup
|
k -> {
|
||||||
) == null) {
|
log.info("Creating a new task group for taskGroupId[%d]", taskGroupId);
|
||||||
sequenceTaskGroup.put(generateSequenceName(taskGroup), taskGroups.get(taskGroupId));
|
return new TaskGroup(
|
||||||
log.info("Created new task group [%d]", taskGroupId);
|
ImmutableMap.copyOf(
|
||||||
}
|
kafkaTask.getIOConfig().getStartPartitions().getPartitionOffsetMap()
|
||||||
|
),
|
||||||
|
kafkaTask.getIOConfig().getMinimumMessageTime(),
|
||||||
|
kafkaTask.getIOConfig().getMaximumMessageTime()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
);
|
||||||
taskGroupsToVerify.add(taskGroupId);
|
taskGroupsToVerify.add(taskGroupId);
|
||||||
taskGroups.get(taskGroupId).tasks.putIfAbsent(taskId, new TaskData());
|
taskGroup.tasks.putIfAbsent(taskId, new TaskData());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return true;
|
return true;
|
||||||
|
@ -1256,7 +1255,6 @@ public class KafkaSupervisor implements Supervisor
|
||||||
// killing all tasks or no task left in the group ?
|
// killing all tasks or no task left in the group ?
|
||||||
// clear state about the taskgroup so that get latest offset information is fetched from metadata store
|
// clear state about the taskgroup so that get latest offset information is fetched from metadata store
|
||||||
log.warn("Clearing task group [%d] information as no valid tasks left the group", groupId);
|
log.warn("Clearing task group [%d] information as no valid tasks left the group", groupId);
|
||||||
sequenceTaskGroup.remove(generateSequenceName(taskGroup));
|
|
||||||
taskGroups.remove(groupId);
|
taskGroups.remove(groupId);
|
||||||
partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET);
|
partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET);
|
||||||
}
|
}
|
||||||
|
@ -1281,9 +1279,10 @@ public class KafkaSupervisor implements Supervisor
|
||||||
Map<Integer, Long> startingPartitions
|
Map<Integer, Long> startingPartitions
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
pendingCompletionTaskGroups.putIfAbsent(groupId, Lists.<TaskGroup>newCopyOnWriteArrayList());
|
final CopyOnWriteArrayList<TaskGroup> taskGroupList = pendingCompletionTaskGroups.computeIfAbsent(
|
||||||
|
groupId,
|
||||||
CopyOnWriteArrayList<TaskGroup> taskGroupList = pendingCompletionTaskGroups.get(groupId);
|
k -> new CopyOnWriteArrayList<>()
|
||||||
|
);
|
||||||
for (TaskGroup taskGroup : taskGroupList) {
|
for (TaskGroup taskGroup : taskGroupList) {
|
||||||
if (taskGroup.partitionOffsets.equals(startingPartitions)) {
|
if (taskGroup.partitionOffsets.equals(startingPartitions)) {
|
||||||
if (taskGroup.tasks.putIfAbsent(taskId, new TaskData()) == null) {
|
if (taskGroup.tasks.putIfAbsent(taskId, new TaskData()) == null) {
|
||||||
|
@ -1411,8 +1410,7 @@ public class KafkaSupervisor implements Supervisor
|
||||||
if (endOffsets != null) {
|
if (endOffsets != null) {
|
||||||
// set a timeout and put this group in pendingCompletionTaskGroups so that it can be monitored for completion
|
// set a timeout and put this group in pendingCompletionTaskGroups so that it can be monitored for completion
|
||||||
group.completionTimeout = DateTimes.nowUtc().plus(ioConfig.getCompletionTimeout());
|
group.completionTimeout = DateTimes.nowUtc().plus(ioConfig.getCompletionTimeout());
|
||||||
pendingCompletionTaskGroups.putIfAbsent(groupId, Lists.<TaskGroup>newCopyOnWriteArrayList());
|
pendingCompletionTaskGroups.computeIfAbsent(groupId, k -> new CopyOnWriteArrayList<>()).add(group);
|
||||||
pendingCompletionTaskGroups.get(groupId).add(group);
|
|
||||||
|
|
||||||
// set endOffsets as the next startOffsets
|
// set endOffsets as the next startOffsets
|
||||||
for (Map.Entry<Integer, Long> entry : endOffsets.entrySet()) {
|
for (Map.Entry<Integer, Long> entry : endOffsets.entrySet()) {
|
||||||
|
@ -1432,7 +1430,6 @@ public class KafkaSupervisor implements Supervisor
|
||||||
partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET);
|
partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET);
|
||||||
}
|
}
|
||||||
|
|
||||||
sequenceTaskGroup.remove(generateSequenceName(group));
|
|
||||||
// remove this task group from the list of current task groups now that it has been handled
|
// remove this task group from the list of current task groups now that it has been handled
|
||||||
taskGroups.remove(groupId);
|
taskGroups.remove(groupId);
|
||||||
}
|
}
|
||||||
|
@ -1456,7 +1453,8 @@ public class KafkaSupervisor implements Supervisor
|
||||||
// metadata store (which will have advanced if we succeeded in publishing and will remain the same if publishing
|
// metadata store (which will have advanced if we succeeded in publishing and will remain the same if publishing
|
||||||
// failed and we need to re-ingest)
|
// failed and we need to re-ingest)
|
||||||
return Futures.transform(
|
return Futures.transform(
|
||||||
stopTasksInGroup(taskGroup), new Function<Object, Map<Integer, Long>>()
|
stopTasksInGroup(taskGroup),
|
||||||
|
new Function<Object, Map<Integer, Long>>()
|
||||||
{
|
{
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
|
@ -1625,15 +1623,15 @@ public class KafkaSupervisor implements Supervisor
|
||||||
log.warn("All tasks in group [%d] failed to publish, killing all tasks for these partitions", groupId);
|
log.warn("All tasks in group [%d] failed to publish, killing all tasks for these partitions", groupId);
|
||||||
} else {
|
} else {
|
||||||
log.makeAlert(
|
log.makeAlert(
|
||||||
"No task in [%s] succeeded before the completion timeout elapsed [%s]!",
|
"No task in [%s] for taskGroup [%d] succeeded before the completion timeout elapsed [%s]!",
|
||||||
group.taskIds(),
|
group.taskIds(),
|
||||||
|
groupId,
|
||||||
ioConfig.getCompletionTimeout()
|
ioConfig.getCompletionTimeout()
|
||||||
).emit();
|
).emit();
|
||||||
}
|
}
|
||||||
|
|
||||||
// reset partitions offsets for this task group so that they will be re-read from metadata storage
|
// reset partitions offsets for this task group so that they will be re-read from metadata storage
|
||||||
partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET);
|
partitionGroups.get(groupId).replaceAll((partition, offset) -> NOT_SET);
|
||||||
sequenceTaskGroup.remove(generateSequenceName(group));
|
|
||||||
// kill all the tasks in this pending completion group
|
// kill all the tasks in this pending completion group
|
||||||
killTasksInGroup(group);
|
killTasksInGroup(group);
|
||||||
// set a flag so the other pending completion groups for this set of partitions will also stop
|
// set a flag so the other pending completion groups for this set of partitions will also stop
|
||||||
|
@ -1693,7 +1691,6 @@ public class KafkaSupervisor implements Supervisor
|
||||||
// be recreated with the next set of offsets
|
// be recreated with the next set of offsets
|
||||||
if (taskData.status.isSuccess()) {
|
if (taskData.status.isSuccess()) {
|
||||||
futures.add(stopTasksInGroup(taskGroup));
|
futures.add(stopTasksInGroup(taskGroup));
|
||||||
sequenceTaskGroup.remove(generateSequenceName(taskGroup));
|
|
||||||
iTaskGroups.remove();
|
iTaskGroups.remove();
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
@ -1735,7 +1732,6 @@ public class KafkaSupervisor implements Supervisor
|
||||||
groupId,
|
groupId,
|
||||||
taskGroup
|
taskGroup
|
||||||
);
|
);
|
||||||
sequenceTaskGroup.put(generateSequenceName(taskGroup), taskGroups.get(groupId));
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1778,6 +1774,7 @@ public class KafkaSupervisor implements Supervisor
|
||||||
DateTime maximumMessageTime = taskGroups.get(groupId).maximumMessageTime.orNull();
|
DateTime maximumMessageTime = taskGroups.get(groupId).maximumMessageTime.orNull();
|
||||||
|
|
||||||
KafkaIOConfig kafkaIOConfig = new KafkaIOConfig(
|
KafkaIOConfig kafkaIOConfig = new KafkaIOConfig(
|
||||||
|
groupId,
|
||||||
sequenceName,
|
sequenceName,
|
||||||
new KafkaPartitions(ioConfig.getTopic(), startPartitions),
|
new KafkaPartitions(ioConfig.getTopic(), startPartitions),
|
||||||
new KafkaPartitions(ioConfig.getTopic(), endPartitions),
|
new KafkaPartitions(ioConfig.getTopic(), endPartitions),
|
||||||
|
@ -1944,7 +1941,7 @@ public class KafkaSupervisor implements Supervisor
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private ListenableFuture<?> stopTasksInGroup(TaskGroup taskGroup)
|
private ListenableFuture<?> stopTasksInGroup(@Nullable TaskGroup taskGroup)
|
||||||
{
|
{
|
||||||
if (taskGroup == null) {
|
if (taskGroup == null) {
|
||||||
return Futures.immediateFuture(null);
|
return Futures.immediateFuture(null);
|
||||||
|
@ -2289,6 +2286,28 @@ public class KafkaSupervisor implements Supervisor
|
||||||
return allStats;
|
return allStats;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
@Nullable
|
||||||
|
TaskGroup removeTaskGroup(int taskGroupId)
|
||||||
|
{
|
||||||
|
return taskGroups.remove(taskGroupId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
void moveTaskGroupToPendingCompletion(int taskGroupId)
|
||||||
|
{
|
||||||
|
final TaskGroup taskGroup = taskGroups.remove(taskGroupId);
|
||||||
|
if (taskGroup != null) {
|
||||||
|
pendingCompletionTaskGroups.computeIfAbsent(taskGroupId, k -> new CopyOnWriteArrayList<>()).add(taskGroup);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
int getNoticesQueueSize()
|
||||||
|
{
|
||||||
|
return notices.size();
|
||||||
|
}
|
||||||
|
|
||||||
private static class StatsFromTaskResult
|
private static class StatsFromTaskResult
|
||||||
{
|
{
|
||||||
private final String groupId;
|
private final String groupId;
|
||||||
|
|
|
@ -50,6 +50,7 @@ public class KafkaIOConfigTest
|
||||||
{
|
{
|
||||||
String jsonStr = "{\n"
|
String jsonStr = "{\n"
|
||||||
+ " \"type\": \"kafka\",\n"
|
+ " \"type\": \"kafka\",\n"
|
||||||
|
+ " \"taskGroupId\": 0,\n"
|
||||||
+ " \"baseSequenceName\": \"my-sequence-name\",\n"
|
+ " \"baseSequenceName\": \"my-sequence-name\",\n"
|
||||||
+ " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n"
|
+ " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n"
|
||||||
+ " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n"
|
+ " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n"
|
||||||
|
@ -82,6 +83,7 @@ public class KafkaIOConfigTest
|
||||||
{
|
{
|
||||||
String jsonStr = "{\n"
|
String jsonStr = "{\n"
|
||||||
+ " \"type\": \"kafka\",\n"
|
+ " \"type\": \"kafka\",\n"
|
||||||
|
+ " \"taskGroupId\": 0,\n"
|
||||||
+ " \"baseSequenceName\": \"my-sequence-name\",\n"
|
+ " \"baseSequenceName\": \"my-sequence-name\",\n"
|
||||||
+ " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n"
|
+ " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n"
|
||||||
+ " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n"
|
+ " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n"
|
||||||
|
@ -118,6 +120,7 @@ public class KafkaIOConfigTest
|
||||||
{
|
{
|
||||||
String jsonStr = "{\n"
|
String jsonStr = "{\n"
|
||||||
+ " \"type\": \"kafka\",\n"
|
+ " \"type\": \"kafka\",\n"
|
||||||
|
+ " \"taskGroupId\": 0,\n"
|
||||||
+ " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n"
|
+ " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n"
|
||||||
+ " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n"
|
+ " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n"
|
||||||
+ " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n"
|
+ " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n"
|
||||||
|
@ -137,6 +140,7 @@ public class KafkaIOConfigTest
|
||||||
{
|
{
|
||||||
String jsonStr = "{\n"
|
String jsonStr = "{\n"
|
||||||
+ " \"type\": \"kafka\",\n"
|
+ " \"type\": \"kafka\",\n"
|
||||||
|
+ " \"taskGroupId\": 0,\n"
|
||||||
+ " \"baseSequenceName\": \"my-sequence-name\",\n"
|
+ " \"baseSequenceName\": \"my-sequence-name\",\n"
|
||||||
+ " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n"
|
+ " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n"
|
||||||
+ " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n"
|
+ " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n"
|
||||||
|
@ -156,6 +160,7 @@ public class KafkaIOConfigTest
|
||||||
{
|
{
|
||||||
String jsonStr = "{\n"
|
String jsonStr = "{\n"
|
||||||
+ " \"type\": \"kafka\",\n"
|
+ " \"type\": \"kafka\",\n"
|
||||||
|
+ " \"taskGroupId\": 0,\n"
|
||||||
+ " \"baseSequenceName\": \"my-sequence-name\",\n"
|
+ " \"baseSequenceName\": \"my-sequence-name\",\n"
|
||||||
+ " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n"
|
+ " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n"
|
||||||
+ " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n"
|
+ " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n"
|
||||||
|
@ -175,6 +180,7 @@ public class KafkaIOConfigTest
|
||||||
{
|
{
|
||||||
String jsonStr = "{\n"
|
String jsonStr = "{\n"
|
||||||
+ " \"type\": \"kafka\",\n"
|
+ " \"type\": \"kafka\",\n"
|
||||||
|
+ " \"taskGroupId\": 0,\n"
|
||||||
+ " \"baseSequenceName\": \"my-sequence-name\",\n"
|
+ " \"baseSequenceName\": \"my-sequence-name\",\n"
|
||||||
+ " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n"
|
+ " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n"
|
||||||
+ " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n"
|
+ " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n"
|
||||||
|
@ -194,6 +200,7 @@ public class KafkaIOConfigTest
|
||||||
{
|
{
|
||||||
String jsonStr = "{\n"
|
String jsonStr = "{\n"
|
||||||
+ " \"type\": \"kafka\",\n"
|
+ " \"type\": \"kafka\",\n"
|
||||||
|
+ " \"taskGroupId\": 0,\n"
|
||||||
+ " \"baseSequenceName\": \"my-sequence-name\",\n"
|
+ " \"baseSequenceName\": \"my-sequence-name\",\n"
|
||||||
+ " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n"
|
+ " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n"
|
||||||
+ " \"endPartitions\": {\"topic\":\"other\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n"
|
+ " \"endPartitions\": {\"topic\":\"other\", \"partitionOffsetMap\" : {\"0\":15, \"1\":200}},\n"
|
||||||
|
@ -214,6 +221,7 @@ public class KafkaIOConfigTest
|
||||||
{
|
{
|
||||||
String jsonStr = "{\n"
|
String jsonStr = "{\n"
|
||||||
+ " \"type\": \"kafka\",\n"
|
+ " \"type\": \"kafka\",\n"
|
||||||
|
+ " \"taskGroupId\": 0,\n"
|
||||||
+ " \"baseSequenceName\": \"my-sequence-name\",\n"
|
+ " \"baseSequenceName\": \"my-sequence-name\",\n"
|
||||||
+ " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n"
|
+ " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n"
|
||||||
+ " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15}},\n"
|
+ " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15}},\n"
|
||||||
|
@ -234,6 +242,7 @@ public class KafkaIOConfigTest
|
||||||
{
|
{
|
||||||
String jsonStr = "{\n"
|
String jsonStr = "{\n"
|
||||||
+ " \"type\": \"kafka\",\n"
|
+ " \"type\": \"kafka\",\n"
|
||||||
|
+ " \"taskGroupId\": 0,\n"
|
||||||
+ " \"baseSequenceName\": \"my-sequence-name\",\n"
|
+ " \"baseSequenceName\": \"my-sequence-name\",\n"
|
||||||
+ " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n"
|
+ " \"startPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":1, \"1\":10}},\n"
|
||||||
+ " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":2}},\n"
|
+ " \"endPartitions\": {\"topic\":\"mytopic\", \"partitionOffsetMap\" : {\"0\":15, \"1\":2}},\n"
|
||||||
|
|
|
@ -262,21 +262,21 @@ public class KafkaIndexTaskTest
|
||||||
private static List<ProducerRecord<byte[], byte[]>> generateRecords(String topic)
|
private static List<ProducerRecord<byte[], byte[]>> generateRecords(String topic)
|
||||||
{
|
{
|
||||||
return ImmutableList.of(
|
return ImmutableList.of(
|
||||||
new ProducerRecord<byte[], byte[]>(topic, 0, null, JB("2008", "a", "y", "10", "20.0", "1.0")),
|
new ProducerRecord<>(topic, 0, null, JB("2008", "a", "y", "10", "20.0", "1.0")),
|
||||||
new ProducerRecord<byte[], byte[]>(topic, 0, null, JB("2009", "b", "y", "10", "20.0", "1.0")),
|
new ProducerRecord<>(topic, 0, null, JB("2009", "b", "y", "10", "20.0", "1.0")),
|
||||||
new ProducerRecord<byte[], byte[]>(topic, 0, null, JB("2010", "c", "y", "10", "20.0", "1.0")),
|
new ProducerRecord<>(topic, 0, null, JB("2010", "c", "y", "10", "20.0", "1.0")),
|
||||||
new ProducerRecord<byte[], byte[]>(topic, 0, null, JB("2011", "d", "y", "10", "20.0", "1.0")),
|
new ProducerRecord<>(topic, 0, null, JB("2011", "d", "y", "10", "20.0", "1.0")),
|
||||||
new ProducerRecord<byte[], byte[]>(topic, 0, null, JB("2011", "e", "y", "10", "20.0", "1.0")),
|
new ProducerRecord<>(topic, 0, null, JB("2011", "e", "y", "10", "20.0", "1.0")),
|
||||||
new ProducerRecord<byte[], byte[]>(topic, 0, null, JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")),
|
new ProducerRecord<>(topic, 0, null, JB("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")),
|
||||||
new ProducerRecord<byte[], byte[]>(topic, 0, null, StringUtils.toUtf8("unparseable")),
|
new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable")),
|
||||||
new ProducerRecord<byte[], byte[]>(topic, 0, null, StringUtils.toUtf8("unparseable2")),
|
new ProducerRecord<>(topic, 0, null, StringUtils.toUtf8("unparseable2")),
|
||||||
new ProducerRecord<byte[], byte[]>(topic, 0, null, null),
|
new ProducerRecord<>(topic, 0, null, null),
|
||||||
new ProducerRecord<byte[], byte[]>(topic, 0, null, JB("2013", "f", "y", "10", "20.0", "1.0")),
|
new ProducerRecord<>(topic, 0, null, JB("2013", "f", "y", "10", "20.0", "1.0")),
|
||||||
new ProducerRecord<byte[], byte[]>(topic, 0, null, JB("2049", "f", "y", "notanumber", "20.0", "1.0")),
|
new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "notanumber", "20.0", "1.0")),
|
||||||
new ProducerRecord<byte[], byte[]>(topic, 0, null, JB("2049", "f", "y", "10", "notanumber", "1.0")),
|
new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "10", "notanumber", "1.0")),
|
||||||
new ProducerRecord<byte[], byte[]>(topic, 0, null, JB("2049", "f", "y", "10", "20.0", "notanumber")),
|
new ProducerRecord<>(topic, 0, null, JB("2049", "f", "y", "10", "20.0", "notanumber")),
|
||||||
new ProducerRecord<byte[], byte[]>(topic, 1, null, JB("2012", "g", "y", "10", "20.0", "1.0")),
|
new ProducerRecord<>(topic, 1, null, JB("2012", "g", "y", "10", "20.0", "1.0")),
|
||||||
new ProducerRecord<byte[], byte[]>(topic, 1, null, JB("2011", "h", "y", "10", "20.0", "1.0"))
|
new ProducerRecord<>(topic, 1, null, JB("2011", "h", "y", "10", "20.0", "1.0"))
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -377,6 +377,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
||||||
|
@ -418,6 +419,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
||||||
|
@ -493,6 +495,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
baseSequenceName,
|
baseSequenceName,
|
||||||
startPartitions,
|
startPartitions,
|
||||||
endPartitions,
|
endPartitions,
|
||||||
|
@ -514,14 +517,16 @@ public class KafkaIndexTaskTest
|
||||||
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
|
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
|
||||||
|
|
||||||
Assert.assertEquals(1, checkpointRequestsHash.size());
|
Assert.assertEquals(1, checkpointRequestsHash.size());
|
||||||
Assert.assertTrue(checkpointRequestsHash.contains(
|
Assert.assertTrue(
|
||||||
Objects.hash(
|
checkpointRequestsHash.contains(
|
||||||
DATA_SCHEMA.getDataSource(),
|
Objects.hash(
|
||||||
baseSequenceName,
|
DATA_SCHEMA.getDataSource(),
|
||||||
new KafkaDataSourceMetadata(startPartitions),
|
0,
|
||||||
new KafkaDataSourceMetadata(new KafkaPartitions(topic, currentOffsets))
|
new KafkaDataSourceMetadata(startPartitions),
|
||||||
|
new KafkaDataSourceMetadata(new KafkaPartitions(topic, currentOffsets))
|
||||||
|
)
|
||||||
)
|
)
|
||||||
));
|
);
|
||||||
|
|
||||||
// Check metrics
|
// Check metrics
|
||||||
Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed());
|
Assert.assertEquals(8, task.getRunner().getRowIngestionMeters().getProcessed());
|
||||||
|
@ -581,6 +586,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
baseSequenceName,
|
baseSequenceName,
|
||||||
startPartitions,
|
startPartitions,
|
||||||
endPartitions,
|
endPartitions,
|
||||||
|
@ -603,14 +609,16 @@ public class KafkaIndexTaskTest
|
||||||
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
|
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
|
||||||
|
|
||||||
Assert.assertEquals(1, checkpointRequestsHash.size());
|
Assert.assertEquals(1, checkpointRequestsHash.size());
|
||||||
Assert.assertTrue(checkpointRequestsHash.contains(
|
Assert.assertTrue(
|
||||||
Objects.hash(
|
checkpointRequestsHash.contains(
|
||||||
DATA_SCHEMA.getDataSource(),
|
Objects.hash(
|
||||||
baseSequenceName,
|
DATA_SCHEMA.getDataSource(),
|
||||||
new KafkaDataSourceMetadata(startPartitions),
|
0,
|
||||||
new KafkaDataSourceMetadata(new KafkaPartitions(topic, checkpoint.getPartitionOffsetMap()))
|
new KafkaDataSourceMetadata(startPartitions),
|
||||||
|
new KafkaDataSourceMetadata(new KafkaPartitions(topic, checkpoint.getPartitionOffsetMap()))
|
||||||
|
)
|
||||||
)
|
)
|
||||||
));
|
);
|
||||||
|
|
||||||
// Check metrics
|
// Check metrics
|
||||||
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getProcessed());
|
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getProcessed());
|
||||||
|
@ -637,6 +645,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 0L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 0L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
||||||
|
@ -690,6 +699,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 0L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 0L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
||||||
|
@ -753,6 +763,7 @@ public class KafkaIndexTaskTest
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 0L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 0L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
||||||
|
@ -812,6 +823,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
|
@ -852,6 +864,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
||||||
|
@ -903,6 +916,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
||||||
|
@ -957,6 +971,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 7L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 7L)),
|
||||||
|
@ -1000,6 +1015,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 13L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 13L)),
|
||||||
|
@ -1081,6 +1097,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 10L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 10L)),
|
||||||
|
@ -1140,6 +1157,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task1 = createTask(
|
final KafkaIndexTask task1 = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
||||||
|
@ -1153,6 +1171,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task2 = createTask(
|
final KafkaIndexTask task2 = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
||||||
|
@ -1206,6 +1225,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task1 = createTask(
|
final KafkaIndexTask task1 = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
||||||
|
@ -1219,6 +1239,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task2 = createTask(
|
final KafkaIndexTask task2 = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
1,
|
||||||
"sequence1",
|
"sequence1",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 3L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 3L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 10L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 10L)),
|
||||||
|
@ -1273,6 +1294,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task1 = createTask(
|
final KafkaIndexTask task1 = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
||||||
|
@ -1286,6 +1308,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task2 = createTask(
|
final KafkaIndexTask task2 = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
1,
|
||||||
"sequence1",
|
"sequence1",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 3L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 3L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 10L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 10L)),
|
||||||
|
@ -1345,6 +1368,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L, 1, 0L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L, 1, 0L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L, 1, 2L)),
|
||||||
|
@ -1409,6 +1433,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task1 = createTask(
|
final KafkaIndexTask task1 = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
||||||
|
@ -1422,6 +1447,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task2 = createTask(
|
final KafkaIndexTask task2 = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
1,
|
||||||
"sequence1",
|
"sequence1",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(1, 0L)),
|
new KafkaPartitions(topic, ImmutableMap.of(1, 0L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(1, 1L)),
|
new KafkaPartitions(topic, ImmutableMap.of(1, 1L)),
|
||||||
|
@ -1477,6 +1503,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task1 = createTask(
|
final KafkaIndexTask task1 = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
||||||
|
@ -1513,6 +1540,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task2 = createTask(
|
final KafkaIndexTask task2 = createTask(
|
||||||
task1.getId(),
|
task1.getId(),
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
||||||
|
@ -1564,6 +1592,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
||||||
|
@ -1647,6 +1676,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 2L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 5L)),
|
||||||
|
@ -1685,6 +1715,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 200L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 200L)),
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 500L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 500L)),
|
||||||
|
@ -1737,6 +1768,7 @@ public class KafkaIndexTaskTest
|
||||||
final KafkaIndexTask task = createTask(
|
final KafkaIndexTask task = createTask(
|
||||||
null,
|
null,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
|
0,
|
||||||
"sequence0",
|
"sequence0",
|
||||||
// task should ignore these and use sequence info sent in the context
|
// task should ignore these and use sequence info sent in the context
|
||||||
new KafkaPartitions(topic, ImmutableMap.of(0, 0L)),
|
new KafkaPartitions(topic, ImmutableMap.of(0, 0L)),
|
||||||
|
@ -2026,18 +2058,20 @@ public class KafkaIndexTaskTest
|
||||||
@Override
|
@Override
|
||||||
public boolean checkPointDataSourceMetadata(
|
public boolean checkPointDataSourceMetadata(
|
||||||
String supervisorId,
|
String supervisorId,
|
||||||
@Nullable String sequenceName,
|
int taskGroupId,
|
||||||
@Nullable DataSourceMetadata previousDataSourceMetadata,
|
@Nullable DataSourceMetadata previousDataSourceMetadata,
|
||||||
@Nullable DataSourceMetadata currentDataSourceMetadata
|
@Nullable DataSourceMetadata currentDataSourceMetadata
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
log.info("Adding checkpoint hash to the set");
|
log.info("Adding checkpoint hash to the set");
|
||||||
checkpointRequestsHash.add(Objects.hash(
|
checkpointRequestsHash.add(
|
||||||
supervisorId,
|
Objects.hash(
|
||||||
sequenceName,
|
supervisorId,
|
||||||
previousDataSourceMetadata,
|
taskGroupId,
|
||||||
currentDataSourceMetadata
|
previousDataSourceMetadata,
|
||||||
));
|
currentDataSourceMetadata
|
||||||
|
)
|
||||||
|
);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package io.druid.indexing.kafka.supervisor;
|
package io.druid.indexing.kafka.supervisor;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
|
@ -61,6 +62,7 @@ import io.druid.java.util.common.StringUtils;
|
||||||
import io.druid.java.util.common.granularity.Granularities;
|
import io.druid.java.util.common.granularity.Granularities;
|
||||||
import io.druid.java.util.common.parsers.JSONPathFieldSpec;
|
import io.druid.java.util.common.parsers.JSONPathFieldSpec;
|
||||||
import io.druid.java.util.common.parsers.JSONPathSpec;
|
import io.druid.java.util.common.parsers.JSONPathSpec;
|
||||||
|
import io.druid.java.util.emitter.EmittingLogger;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import io.druid.segment.TestHelper;
|
import io.druid.segment.TestHelper;
|
||||||
|
@ -70,6 +72,7 @@ import io.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
import io.druid.segment.realtime.FireDepartment;
|
import io.druid.segment.realtime.FireDepartment;
|
||||||
import io.druid.server.metrics.DruidMonitorSchedulerConfig;
|
import io.druid.server.metrics.DruidMonitorSchedulerConfig;
|
||||||
import io.druid.server.metrics.NoopServiceEmitter;
|
import io.druid.server.metrics.NoopServiceEmitter;
|
||||||
|
import io.druid.server.metrics.ExceptionCapturingServiceEmitter;
|
||||||
import org.apache.curator.test.TestingCluster;
|
import org.apache.curator.test.TestingCluster;
|
||||||
import org.apache.kafka.clients.producer.KafkaProducer;
|
import org.apache.kafka.clients.producer.KafkaProducer;
|
||||||
import org.apache.kafka.clients.producer.ProducerRecord;
|
import org.apache.kafka.clients.producer.ProducerRecord;
|
||||||
|
@ -99,7 +102,9 @@ import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.Executor;
|
import java.util.concurrent.Executor;
|
||||||
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
import static org.easymock.EasyMock.anyBoolean;
|
import static org.easymock.EasyMock.anyBoolean;
|
||||||
import static org.easymock.EasyMock.anyObject;
|
import static org.easymock.EasyMock.anyObject;
|
||||||
|
@ -141,6 +146,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
private TaskQueue taskQueue;
|
private TaskQueue taskQueue;
|
||||||
private String topic;
|
private String topic;
|
||||||
private RowIngestionMetersFactory rowIngestionMetersFactory;
|
private RowIngestionMetersFactory rowIngestionMetersFactory;
|
||||||
|
private ExceptionCapturingServiceEmitter serviceEmitter;
|
||||||
|
|
||||||
private static String getTopic()
|
private static String getTopic()
|
||||||
{
|
{
|
||||||
|
@ -213,6 +219,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
|
|
||||||
topic = getTopic();
|
topic = getTopic();
|
||||||
rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory();
|
rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory();
|
||||||
|
serviceEmitter = new ExceptionCapturingServiceEmitter();
|
||||||
|
EmittingLogger.registerEmitter(serviceEmitter);
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
@ -553,7 +561,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id1 = createKafkaIndexTask(
|
Task id1 = createKafkaIndexTask(
|
||||||
"id1",
|
"id1",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"index_kafka_testDS__some_other_sequenceName",
|
1,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 0L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 0L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 10L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 10L)),
|
||||||
null,
|
null,
|
||||||
|
@ -564,7 +572,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id2 = createKafkaIndexTask(
|
Task id2 = createKafkaIndexTask(
|
||||||
"id2",
|
"id2",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 333L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 333L)),
|
||||||
null,
|
null,
|
||||||
|
@ -575,7 +583,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id3 = createKafkaIndexTask(
|
Task id3 = createKafkaIndexTask(
|
||||||
"id3",
|
"id3",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"index_kafka_testDS__some_other_sequenceName",
|
1,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 1L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 1L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 330L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 333L, 1, 333L, 2, 330L)),
|
||||||
null,
|
null,
|
||||||
|
@ -586,7 +594,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id4 = createKafkaIndexTask(
|
Task id4 = createKafkaIndexTask(
|
||||||
"id4",
|
"id4",
|
||||||
"other-datasource",
|
"other-datasource",
|
||||||
"index_kafka_testDS_d927edff33c4b3f",
|
2,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 0L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 0L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 10L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 10L)),
|
||||||
null,
|
null,
|
||||||
|
@ -634,7 +642,9 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
|
|
||||||
TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
||||||
checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L));
|
checkpoints.put(0, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L));
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(2);
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints))
|
||||||
|
.times(2);
|
||||||
|
|
||||||
replayAll();
|
replayAll();
|
||||||
|
|
||||||
|
@ -652,7 +662,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id1 = createKafkaIndexTask(
|
Task id1 = createKafkaIndexTask(
|
||||||
"id1",
|
"id1",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -661,7 +671,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id2 = createKafkaIndexTask(
|
Task id2 = createKafkaIndexTask(
|
||||||
"id2",
|
"id2",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-1",
|
1,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(1, 0L)),
|
new KafkaPartitions("topic", ImmutableMap.of(1, 0L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(1, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(1, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -670,7 +680,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id3 = createKafkaIndexTask(
|
Task id3 = createKafkaIndexTask(
|
||||||
"id3",
|
"id3",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -679,7 +689,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id4 = createKafkaIndexTask(
|
Task id4 = createKafkaIndexTask(
|
||||||
"id4",
|
"id4",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -688,7 +698,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id5 = createKafkaIndexTask(
|
Task id5 = createKafkaIndexTask(
|
||||||
"id5",
|
"id5",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(1, 0L, 2, 0L)),
|
new KafkaPartitions("topic", ImmutableMap.of(1, 0L, 2, 0L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -727,8 +737,12 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
|
checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
|
||||||
TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
|
TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
|
||||||
checkpoints2.put(0, ImmutableMap.of(1, 0L));
|
checkpoints2.put(0, ImmutableMap.of(1, 0L));
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints1)).times(1);
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean()))
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints2)).times(1);
|
.andReturn(Futures.immediateFuture(checkpoints1))
|
||||||
|
.times(1);
|
||||||
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints2))
|
||||||
|
.times(1);
|
||||||
|
|
||||||
taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
|
taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
|
||||||
taskQueue.shutdown("id4");
|
taskQueue.shutdown("id4");
|
||||||
|
@ -765,10 +779,12 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
|
checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
|
||||||
TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
|
TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
|
||||||
checkpoints2.put(0, ImmutableMap.of(1, 0L));
|
checkpoints2.put(0, ImmutableMap.of(1, 0L));
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints1))
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean()))
|
||||||
.anyTimes();
|
.andReturn(Futures.immediateFuture(checkpoints1))
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints2))
|
.anyTimes();
|
||||||
.anyTimes();
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints2))
|
||||||
|
.anyTimes();
|
||||||
|
|
||||||
taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
|
taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
|
||||||
replayAll();
|
replayAll();
|
||||||
|
@ -830,7 +846,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id1 = createKafkaIndexTask(
|
Task id1 = createKafkaIndexTask(
|
||||||
"id1",
|
"id1",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
now,
|
now,
|
||||||
|
@ -857,7 +873,9 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
|
|
||||||
TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
||||||
checkpoints.put(0, ImmutableMap.of(0, 0L, 2, 0L));
|
checkpoints.put(0, ImmutableMap.of(0, 0L, 2, 0L));
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(2);
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints))
|
||||||
|
.times(2);
|
||||||
|
|
||||||
taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
|
taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
|
||||||
replayAll();
|
replayAll();
|
||||||
|
@ -878,9 +896,12 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
reset(taskClient);
|
reset(taskClient);
|
||||||
|
|
||||||
// for the newly created replica task
|
// for the newly created replica task
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints))
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean()))
|
||||||
.times(2);
|
.andReturn(Futures.immediateFuture(checkpoints))
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(1);
|
.times(2);
|
||||||
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints))
|
||||||
|
.times(1);
|
||||||
|
|
||||||
expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(captured.getValue())).anyTimes();
|
expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(captured.getValue())).anyTimes();
|
||||||
expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId())));
|
expect(taskStorage.getStatus(iHaveFailed.getId())).andReturn(Optional.of(TaskStatus.failure(iHaveFailed.getId())));
|
||||||
|
@ -953,10 +974,12 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
|
TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
|
||||||
checkpoints2.put(0, ImmutableMap.of(1, 0L));
|
checkpoints2.put(0, ImmutableMap.of(1, 0L));
|
||||||
// there would be 4 tasks, 2 for each task group
|
// there would be 4 tasks, 2 for each task group
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints1))
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean()))
|
||||||
.times(2);
|
.andReturn(Futures.immediateFuture(checkpoints1))
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints2))
|
.times(2);
|
||||||
.times(2);
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints2))
|
||||||
|
.times(2);
|
||||||
|
|
||||||
expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes();
|
expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes();
|
||||||
for (Task task : tasks) {
|
for (Task task : tasks) {
|
||||||
|
@ -1063,10 +1086,12 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
|
checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
|
||||||
TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
|
TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
|
||||||
checkpoints2.put(0, ImmutableMap.of(1, 0L));
|
checkpoints2.put(0, ImmutableMap.of(1, 0L));
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints1))
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean()))
|
||||||
.times(2);
|
.andReturn(Futures.immediateFuture(checkpoints1))
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints2))
|
.times(2);
|
||||||
.times(2);
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints2))
|
||||||
|
.times(2);
|
||||||
|
|
||||||
replay(taskStorage, taskRunner, taskClient, taskQueue);
|
replay(taskStorage, taskRunner, taskClient, taskQueue);
|
||||||
|
|
||||||
|
@ -1100,7 +1125,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task task = createKafkaIndexTask(
|
Task task = createKafkaIndexTask(
|
||||||
"id1",
|
"id1",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -1192,7 +1217,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task task = createKafkaIndexTask(
|
Task task = createKafkaIndexTask(
|
||||||
"id1",
|
"id1",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 2, 0L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -1282,7 +1307,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id1 = createKafkaIndexTask(
|
Task id1 = createKafkaIndexTask(
|
||||||
"id1",
|
"id1",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -1292,7 +1317,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id2 = createKafkaIndexTask(
|
Task id2 = createKafkaIndexTask(
|
||||||
"id2",
|
"id2",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 1L, 1, 2L, 2, 3L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -1330,7 +1355,9 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
// since id1 is publishing, so getCheckpoints wouldn't be called for it
|
// since id1 is publishing, so getCheckpoints wouldn't be called for it
|
||||||
TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
||||||
checkpoints.put(0, ImmutableMap.of(0, 1L, 1, 2L, 2, 3L));
|
checkpoints.put(0, ImmutableMap.of(0, 1L, 1, 2L, 2, 3L));
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(1);
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints))
|
||||||
|
.times(1);
|
||||||
|
|
||||||
replayAll();
|
replayAll();
|
||||||
|
|
||||||
|
@ -1404,10 +1431,12 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
|
checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
|
||||||
TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
|
TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
|
||||||
checkpoints2.put(0, ImmutableMap.of(1, 0L));
|
checkpoints2.put(0, ImmutableMap.of(1, 0L));
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints1))
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean()))
|
||||||
.times(2);
|
.andReturn(Futures.immediateFuture(checkpoints1))
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints2))
|
.times(2);
|
||||||
.times(2);
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints2))
|
||||||
|
.times(2);
|
||||||
|
|
||||||
expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes();
|
expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes();
|
||||||
for (Task task : tasks) {
|
for (Task task : tasks) {
|
||||||
|
@ -1463,10 +1492,12 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
|
checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
|
||||||
TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
|
TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
|
||||||
checkpoints2.put(0, ImmutableMap.of(1, 0L));
|
checkpoints2.put(0, ImmutableMap.of(1, 0L));
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints1))
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean()))
|
||||||
.times(2);
|
.andReturn(Futures.immediateFuture(checkpoints1))
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints2))
|
.times(2);
|
||||||
.times(2);
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints2))
|
||||||
|
.times(2);
|
||||||
|
|
||||||
captured = Capture.newInstance(CaptureType.ALL);
|
captured = Capture.newInstance(CaptureType.ALL);
|
||||||
expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes();
|
expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes();
|
||||||
|
@ -1540,10 +1571,12 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
|
checkpoints1.put(0, ImmutableMap.of(0, 0L, 2, 0L));
|
||||||
TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
|
TreeMap<Integer, Map<Integer, Long>> checkpoints2 = new TreeMap<>();
|
||||||
checkpoints2.put(0, ImmutableMap.of(1, 0L));
|
checkpoints2.put(0, ImmutableMap.of(1, 0L));
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints1))
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-0"), anyBoolean()))
|
||||||
.times(2);
|
.andReturn(Futures.immediateFuture(checkpoints1))
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints2))
|
.times(2);
|
||||||
.times(2);
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("sequenceName-1"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints2))
|
||||||
|
.times(2);
|
||||||
|
|
||||||
captured = Capture.newInstance(CaptureType.ALL);
|
captured = Capture.newInstance(CaptureType.ALL);
|
||||||
expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes();
|
expect(taskStorage.getActiveTasks()).andReturn(tasks).anyTimes();
|
||||||
|
@ -1622,7 +1655,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id1 = createKafkaIndexTask(
|
Task id1 = createKafkaIndexTask(
|
||||||
"id1",
|
"id1",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -1632,7 +1665,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id2 = createKafkaIndexTask(
|
Task id2 = createKafkaIndexTask(
|
||||||
"id2",
|
"id2",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -1642,7 +1675,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id3 = createKafkaIndexTask(
|
Task id3 = createKafkaIndexTask(
|
||||||
"id3",
|
"id3",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -1678,8 +1711,12 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
// getCheckpoints will not be called for id1 as it is in publishing state
|
// getCheckpoints will not be called for id1 as it is in publishing state
|
||||||
TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
||||||
checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
|
checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(1);
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean()))
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(1);
|
.andReturn(Futures.immediateFuture(checkpoints))
|
||||||
|
.times(1);
|
||||||
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints))
|
||||||
|
.times(1);
|
||||||
|
|
||||||
taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
|
taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
|
||||||
replayAll();
|
replayAll();
|
||||||
|
@ -1824,7 +1861,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id1 = createKafkaIndexTask(
|
Task id1 = createKafkaIndexTask(
|
||||||
"id1",
|
"id1",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -1834,7 +1871,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id2 = createKafkaIndexTask(
|
Task id2 = createKafkaIndexTask(
|
||||||
"id2",
|
"id2",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -1844,7 +1881,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id3 = createKafkaIndexTask(
|
Task id3 = createKafkaIndexTask(
|
||||||
"id3",
|
"id3",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -1879,8 +1916,12 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
|
|
||||||
TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
||||||
checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
|
checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(1);
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean()))
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(1);
|
.andReturn(Futures.immediateFuture(checkpoints))
|
||||||
|
.times(1);
|
||||||
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints))
|
||||||
|
.times(1);
|
||||||
|
|
||||||
taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
|
taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
|
||||||
replayAll();
|
replayAll();
|
||||||
|
@ -1908,7 +1949,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id1 = createKafkaIndexTask(
|
Task id1 = createKafkaIndexTask(
|
||||||
"id1",
|
"id1",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -1918,7 +1959,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id2 = createKafkaIndexTask(
|
Task id2 = createKafkaIndexTask(
|
||||||
"id2",
|
"id2",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -1928,7 +1969,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
Task id3 = createKafkaIndexTask(
|
Task id3 = createKafkaIndexTask(
|
||||||
"id3",
|
"id3",
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
"sequenceName-0",
|
0,
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
||||||
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
new KafkaPartitions("topic", ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
null,
|
null,
|
||||||
|
@ -1958,9 +1999,15 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
|
|
||||||
TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
||||||
checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
|
checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(1);
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean()))
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(1);
|
.andReturn(Futures.immediateFuture(checkpoints))
|
||||||
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean())).andReturn(Futures.immediateFuture(checkpoints)).times(1);
|
.times(1);
|
||||||
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints))
|
||||||
|
.times(1);
|
||||||
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints))
|
||||||
|
.times(1);
|
||||||
|
|
||||||
taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
|
taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
|
||||||
replayAll();
|
replayAll();
|
||||||
|
@ -1980,6 +2027,172 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
verifyAll();
|
verifyAll();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 60_000L)
|
||||||
|
public void testCheckpointForInactiveTaskGroup()
|
||||||
|
throws InterruptedException, ExecutionException, TimeoutException, JsonProcessingException
|
||||||
|
{
|
||||||
|
supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false);
|
||||||
|
//not adding any events
|
||||||
|
final Task id1 = createKafkaIndexTask(
|
||||||
|
"id1",
|
||||||
|
DATASOURCE,
|
||||||
|
0,
|
||||||
|
new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
||||||
|
new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
|
||||||
|
final Task id2 = createKafkaIndexTask(
|
||||||
|
"id2",
|
||||||
|
DATASOURCE,
|
||||||
|
0,
|
||||||
|
new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
||||||
|
new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
|
||||||
|
final Task id3 = createKafkaIndexTask(
|
||||||
|
"id3",
|
||||||
|
DATASOURCE,
|
||||||
|
0,
|
||||||
|
new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
||||||
|
new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
|
||||||
|
expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
|
||||||
|
expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
|
||||||
|
expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes();
|
||||||
|
expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes();
|
||||||
|
expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes();
|
||||||
|
expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes();
|
||||||
|
expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
|
||||||
|
expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
|
||||||
|
expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
|
||||||
|
expect(
|
||||||
|
indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KafkaDataSourceMetadata(null)
|
||||||
|
).anyTimes();
|
||||||
|
expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
|
||||||
|
expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
|
||||||
|
expect(taskClient.getStatusAsync("id3")).andReturn(Futures.immediateFuture(KafkaIndexTask.Status.READING));
|
||||||
|
|
||||||
|
final DateTime startTime = DateTimes.nowUtc();
|
||||||
|
expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(startTime));
|
||||||
|
expect(taskClient.getStartTimeAsync("id2")).andReturn(Futures.immediateFuture(startTime));
|
||||||
|
expect(taskClient.getStartTimeAsync("id3")).andReturn(Futures.immediateFuture(startTime));
|
||||||
|
|
||||||
|
final TreeMap<Integer, Map<Integer, Long>> checkpoints = new TreeMap<>();
|
||||||
|
checkpoints.put(0, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L));
|
||||||
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id1"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints))
|
||||||
|
.times(1);
|
||||||
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id2"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints))
|
||||||
|
.times(1);
|
||||||
|
expect(taskClient.getCheckpointsAsync(EasyMock.contains("id3"), anyBoolean()))
|
||||||
|
.andReturn(Futures.immediateFuture(checkpoints))
|
||||||
|
.times(1);
|
||||||
|
|
||||||
|
taskRunner.registerListener(anyObject(TaskRunnerListener.class), anyObject(Executor.class));
|
||||||
|
replayAll();
|
||||||
|
|
||||||
|
supervisor.start();
|
||||||
|
supervisor.runInternal();
|
||||||
|
|
||||||
|
final Map<Integer, Long> fakeCheckpoints = Collections.emptyMap();
|
||||||
|
supervisor.moveTaskGroupToPendingCompletion(0);
|
||||||
|
supervisor.checkpoint(
|
||||||
|
0,
|
||||||
|
new KafkaDataSourceMetadata(new KafkaPartitions(topic, checkpoints.get(0))),
|
||||||
|
new KafkaDataSourceMetadata(new KafkaPartitions(topic, fakeCheckpoints))
|
||||||
|
);
|
||||||
|
|
||||||
|
while (supervisor.getNoticesQueueSize() > 0) {
|
||||||
|
Thread.sleep(100);
|
||||||
|
}
|
||||||
|
|
||||||
|
verifyAll();
|
||||||
|
|
||||||
|
Assert.assertNull(serviceEmitter.getStackTrace());
|
||||||
|
Assert.assertNull(serviceEmitter.getExceptionMessage());
|
||||||
|
Assert.assertNull(serviceEmitter.getExceptionClass());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 60_000L)
|
||||||
|
public void testCheckpointForUnknownTaskGroup() throws InterruptedException
|
||||||
|
{
|
||||||
|
supervisor = getSupervisor(2, 1, true, "PT1S", null, null, false);
|
||||||
|
//not adding any events
|
||||||
|
final Task id1 = createKafkaIndexTask(
|
||||||
|
"id1",
|
||||||
|
DATASOURCE,
|
||||||
|
0,
|
||||||
|
new KafkaPartitions(topic, ImmutableMap.of(0, 0L, 1, 0L, 2, 0L)),
|
||||||
|
new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
|
||||||
|
final Task id2 = createKafkaIndexTask(
|
||||||
|
"id2",
|
||||||
|
DATASOURCE,
|
||||||
|
0,
|
||||||
|
new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
||||||
|
new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
|
||||||
|
final Task id3 = createKafkaIndexTask(
|
||||||
|
"id3",
|
||||||
|
DATASOURCE,
|
||||||
|
0,
|
||||||
|
new KafkaPartitions(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L)),
|
||||||
|
new KafkaPartitions(topic, ImmutableMap.of(0, Long.MAX_VALUE, 1, Long.MAX_VALUE, 2, Long.MAX_VALUE)),
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
|
||||||
|
expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
|
||||||
|
expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
|
||||||
|
expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of(id1, id2, id3)).anyTimes();
|
||||||
|
expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes();
|
||||||
|
expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes();
|
||||||
|
expect(taskStorage.getStatus("id3")).andReturn(Optional.of(TaskStatus.running("id3"))).anyTimes();
|
||||||
|
expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
|
||||||
|
expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
|
||||||
|
expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
|
||||||
|
expect(
|
||||||
|
indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KafkaDataSourceMetadata(null)
|
||||||
|
).anyTimes();
|
||||||
|
|
||||||
|
replayAll();
|
||||||
|
|
||||||
|
supervisor.start();
|
||||||
|
|
||||||
|
supervisor.checkpoint(
|
||||||
|
0,
|
||||||
|
new KafkaDataSourceMetadata(new KafkaPartitions(topic, Collections.emptyMap())),
|
||||||
|
new KafkaDataSourceMetadata(new KafkaPartitions(topic, Collections.emptyMap()))
|
||||||
|
);
|
||||||
|
|
||||||
|
while (supervisor.getNoticesQueueSize() > 0) {
|
||||||
|
Thread.sleep(100);
|
||||||
|
}
|
||||||
|
|
||||||
|
verifyAll();
|
||||||
|
|
||||||
|
Assert.assertNotNull(serviceEmitter.getStackTrace());
|
||||||
|
Assert.assertEquals(
|
||||||
|
"WTH?! cannot find taskGroup [0] among all taskGroups [{}]",
|
||||||
|
serviceEmitter.getExceptionMessage()
|
||||||
|
);
|
||||||
|
Assert.assertEquals(ISE.class, serviceEmitter.getExceptionClass());
|
||||||
|
}
|
||||||
|
|
||||||
private void addSomeEvents(int numEventsPerPartition) throws Exception
|
private void addSomeEvents(int numEventsPerPartition) throws Exception
|
||||||
{
|
{
|
||||||
try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
|
try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
|
||||||
|
@ -2106,7 +2319,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
private KafkaIndexTask createKafkaIndexTask(
|
private KafkaIndexTask createKafkaIndexTask(
|
||||||
String id,
|
String id,
|
||||||
String dataSource,
|
String dataSource,
|
||||||
String sequenceName,
|
int taskGroupId,
|
||||||
KafkaPartitions startPartitions,
|
KafkaPartitions startPartitions,
|
||||||
KafkaPartitions endPartitions,
|
KafkaPartitions endPartitions,
|
||||||
DateTime minimumMessageTime,
|
DateTime minimumMessageTime,
|
||||||
|
@ -2119,7 +2332,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
getDataSchema(dataSource),
|
getDataSchema(dataSource),
|
||||||
tuningConfig,
|
tuningConfig,
|
||||||
new KafkaIOConfig(
|
new KafkaIOConfig(
|
||||||
sequenceName,
|
taskGroupId,
|
||||||
|
"sequenceName-" + taskGroupId,
|
||||||
startPartitions,
|
startPartitions,
|
||||||
endPartitions,
|
endPartitions,
|
||||||
ImmutableMap.<String, String>of(),
|
ImmutableMap.<String, String>of(),
|
||||||
|
@ -2128,7 +2342,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
||||||
maximumMessageTime,
|
maximumMessageTime,
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
ImmutableMap.<String, Object>of(),
|
Collections.emptyMap(),
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
rowIngestionMetersFactory
|
rowIngestionMetersFactory
|
||||||
|
|
|
@ -21,27 +21,28 @@ package io.druid.indexing.common.actions;
|
||||||
|
|
||||||
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.base.Preconditions;
|
||||||
import io.druid.indexing.common.task.Task;
|
import io.druid.indexing.common.task.Task;
|
||||||
import io.druid.indexing.overlord.DataSourceMetadata;
|
import io.druid.indexing.overlord.DataSourceMetadata;
|
||||||
|
|
||||||
public class CheckPointDataSourceMetadataAction implements TaskAction<Boolean>
|
public class CheckPointDataSourceMetadataAction implements TaskAction<Boolean>
|
||||||
{
|
{
|
||||||
private final String supervisorId;
|
private final String supervisorId;
|
||||||
private final String sequenceName;
|
private final int taskGroupId;
|
||||||
private final DataSourceMetadata previousCheckPoint;
|
private final DataSourceMetadata previousCheckPoint;
|
||||||
private final DataSourceMetadata currentCheckPoint;
|
private final DataSourceMetadata currentCheckPoint;
|
||||||
|
|
||||||
public CheckPointDataSourceMetadataAction(
|
public CheckPointDataSourceMetadataAction(
|
||||||
@JsonProperty("supervisorId") String supervisorId,
|
@JsonProperty("supervisorId") String supervisorId,
|
||||||
@JsonProperty("sequenceName") String sequenceName,
|
@JsonProperty("taskGroupId") Integer taskGroupId,
|
||||||
@JsonProperty("previousCheckPoint") DataSourceMetadata previousCheckPoint,
|
@JsonProperty("previousCheckPoint") DataSourceMetadata previousCheckPoint,
|
||||||
@JsonProperty("currentCheckPoint") DataSourceMetadata currentCheckPoint
|
@JsonProperty("currentCheckPoint") DataSourceMetadata currentCheckPoint
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.supervisorId = supervisorId;
|
this.supervisorId = Preconditions.checkNotNull(supervisorId, "supervisorId");
|
||||||
this.sequenceName = sequenceName;
|
this.taskGroupId = Preconditions.checkNotNull(taskGroupId, "taskGroupId");
|
||||||
this.previousCheckPoint = previousCheckPoint;
|
this.previousCheckPoint = Preconditions.checkNotNull(previousCheckPoint, "previousCheckPoint");
|
||||||
this.currentCheckPoint = currentCheckPoint;
|
this.currentCheckPoint = Preconditions.checkNotNull(currentCheckPoint, "currentCheckPoint");
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -51,9 +52,9 @@ public class CheckPointDataSourceMetadataAction implements TaskAction<Boolean>
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public String getSequenceName()
|
public int getTaskGroupId()
|
||||||
{
|
{
|
||||||
return sequenceName;
|
return taskGroupId;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -81,8 +82,12 @@ public class CheckPointDataSourceMetadataAction implements TaskAction<Boolean>
|
||||||
Task task, TaskActionToolbox toolbox
|
Task task, TaskActionToolbox toolbox
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return toolbox.getSupervisorManager()
|
return toolbox.getSupervisorManager().checkPointDataSourceMetadata(
|
||||||
.checkPointDataSourceMetadata(supervisorId, sequenceName, previousCheckPoint, currentCheckPoint);
|
supervisorId,
|
||||||
|
taskGroupId,
|
||||||
|
previousCheckPoint,
|
||||||
|
currentCheckPoint
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -96,7 +101,7 @@ public class CheckPointDataSourceMetadataAction implements TaskAction<Boolean>
|
||||||
{
|
{
|
||||||
return "CheckPointDataSourceMetadataAction{" +
|
return "CheckPointDataSourceMetadataAction{" +
|
||||||
"supervisorId='" + supervisorId + '\'' +
|
"supervisorId='" + supervisorId + '\'' +
|
||||||
", sequenceName='" + sequenceName + '\'' +
|
", taskGroupId='" + taskGroupId + '\'' +
|
||||||
", previousCheckPoint=" + previousCheckPoint +
|
", previousCheckPoint=" + previousCheckPoint +
|
||||||
", currentCheckPoint=" + currentCheckPoint +
|
", currentCheckPoint=" + currentCheckPoint +
|
||||||
'}';
|
'}';
|
||||||
|
|
|
@ -165,9 +165,9 @@ public class SupervisorManager
|
||||||
|
|
||||||
public boolean checkPointDataSourceMetadata(
|
public boolean checkPointDataSourceMetadata(
|
||||||
String supervisorId,
|
String supervisorId,
|
||||||
@Nullable String sequenceName,
|
int taskGroupId,
|
||||||
@Nullable DataSourceMetadata previousDataSourceMetadata,
|
DataSourceMetadata previousDataSourceMetadata,
|
||||||
@Nullable DataSourceMetadata currentDataSourceMetadata
|
DataSourceMetadata currentDataSourceMetadata
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
|
@ -178,7 +178,7 @@ public class SupervisorManager
|
||||||
|
|
||||||
Preconditions.checkNotNull(supervisor, "supervisor could not be found");
|
Preconditions.checkNotNull(supervisor, "supervisor could not be found");
|
||||||
|
|
||||||
supervisor.lhs.checkpoint(sequenceName, previousDataSourceMetadata, currentDataSourceMetadata);
|
supervisor.lhs.checkpoint(taskGroupId, previousDataSourceMetadata, currentDataSourceMetadata);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
|
|
|
@ -35,6 +35,10 @@ import java.io.StringWriter;
|
||||||
*/
|
*/
|
||||||
public class EmittingLogger extends Logger
|
public class EmittingLogger extends Logger
|
||||||
{
|
{
|
||||||
|
public static final String EXCEPTION_TYPE_KEY = "exceptionType";
|
||||||
|
public static final String EXCEPTION_MESSAGE_KEY = "exceptionMessage";
|
||||||
|
public static final String EXCEPTION_STACK_TRACE_KEY = "exceptionStackTrace";
|
||||||
|
|
||||||
private static volatile ServiceEmitter emitter = null;
|
private static volatile ServiceEmitter emitter = null;
|
||||||
|
|
||||||
private final String className;
|
private final String className;
|
||||||
|
|
|
@ -83,9 +83,9 @@ public class NoopSupervisorSpec implements SupervisorSpec
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void checkpoint(
|
public void checkpoint(
|
||||||
@Nullable String sequenceName,
|
int taskGroupId,
|
||||||
@Nullable DataSourceMetadata previousCheckPoint,
|
DataSourceMetadata previousCheckPoint,
|
||||||
@Nullable DataSourceMetadata currentCheckPoint
|
DataSourceMetadata currentCheckPoint
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
|
||||||
|
|
|
@ -22,7 +22,6 @@ package io.druid.indexing.overlord.supervisor;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import io.druid.indexing.overlord.DataSourceMetadata;
|
import io.druid.indexing.overlord.DataSourceMetadata;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
public interface Supervisor
|
public interface Supervisor
|
||||||
|
@ -52,13 +51,9 @@ public interface Supervisor
|
||||||
* for example - Kafka Supervisor uses this to merge and handoff segments containing at least the data
|
* for example - Kafka Supervisor uses this to merge and handoff segments containing at least the data
|
||||||
* represented by {@param currentCheckpoint} DataSourceMetadata
|
* represented by {@param currentCheckpoint} DataSourceMetadata
|
||||||
*
|
*
|
||||||
* @param sequenceName unique Identifier to figure out for which sequence to do checkpointing
|
* @param taskGroupId unique Identifier to figure out for which sequence to do checkpointing
|
||||||
* @param previousCheckPoint DataSourceMetadata checkpointed in previous call
|
* @param previousCheckPoint DataSourceMetadata checkpointed in previous call
|
||||||
* @param currentCheckPoint current DataSourceMetadata to be checkpointed
|
* @param currentCheckPoint current DataSourceMetadata to be checkpointed
|
||||||
*/
|
*/
|
||||||
void checkpoint(
|
void checkpoint(int taskGroupId, DataSourceMetadata previousCheckPoint, DataSourceMetadata currentCheckPoint);
|
||||||
@Nullable String sequenceName,
|
|
||||||
@Nullable DataSourceMetadata previousCheckPoint,
|
|
||||||
@Nullable DataSourceMetadata currentCheckPoint
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,71 @@
|
||||||
|
/*
|
||||||
|
* 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 io.druid.server.metrics;
|
||||||
|
|
||||||
|
import io.druid.java.util.emitter.EmittingLogger;
|
||||||
|
import io.druid.java.util.emitter.core.Event;
|
||||||
|
import io.druid.java.util.emitter.service.ServiceEmitter;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class ExceptionCapturingServiceEmitter extends ServiceEmitter
|
||||||
|
{
|
||||||
|
private volatile Class exceptionClass;
|
||||||
|
private volatile String exceptionMessage;
|
||||||
|
private volatile String stackTrace;
|
||||||
|
|
||||||
|
public ExceptionCapturingServiceEmitter()
|
||||||
|
{
|
||||||
|
super("", "", null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void emit(Event event)
|
||||||
|
{
|
||||||
|
//noinspection unchecked
|
||||||
|
final Map<String, Object> dataMap = (Map<String, Object>) event.toMap().get("data");
|
||||||
|
final Class exceptionClass = (Class) dataMap.get(EmittingLogger.EXCEPTION_TYPE_KEY);
|
||||||
|
if (exceptionClass != null) {
|
||||||
|
final String exceptionMessage = (String) dataMap.get(EmittingLogger.EXCEPTION_MESSAGE_KEY);
|
||||||
|
final String stackTrace = (String) dataMap.get(EmittingLogger.EXCEPTION_STACK_TRACE_KEY);
|
||||||
|
this.exceptionClass = exceptionClass;
|
||||||
|
this.exceptionMessage = exceptionMessage;
|
||||||
|
this.stackTrace = stackTrace;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
public Class getExceptionClass()
|
||||||
|
{
|
||||||
|
return exceptionClass;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
public String getExceptionMessage()
|
||||||
|
{
|
||||||
|
return exceptionMessage;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
public String getStackTrace()
|
||||||
|
{
|
||||||
|
return stackTrace;
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue